) accu.get();
+ // TODO: Get num_row_groups from native
+ // intAccum.add(fileReader.getRowGroups().size());
+ }
+ }
+
+ this.handle =
+ Native.initRecordBatchReader(
+ filePath, fileSize, start, length, serializedRequestedArrowSchema, timeZoneId);
+ isInitialized = true;
+ }
+
+ public void setSparkSchema(StructType schema) {
+ this.sparkSchema = schema;
+ }
+
+ public AbstractColumnReader[] getColumnReaders() {
+ return columnReaders;
+ }
+
+ @Override
+ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+ throws IOException, InterruptedException {
+ // Do nothing. The initialization work is done in 'init' already.
+ }
+
+ @Override
+ public boolean nextKeyValue() throws IOException {
+ return nextBatch();
+ }
+
+ @Override
+ public Void getCurrentKey() {
+ return null;
+ }
+
+ @Override
+ public ColumnarBatch getCurrentValue() {
+ return currentBatch();
+ }
+
+ @Override
+ public float getProgress() {
+ return 0;
+ }
+
+ /**
+ * Returns the current columnar batch being read.
+ *
+ * Note that this must be called AFTER {@link NativeBatchReader#nextBatch()}.
+ */
+ public ColumnarBatch currentBatch() {
+ return currentBatch;
+ }
+
+ /**
+ * Loads the next batch of rows. This is called by Spark _and_ Iceberg
+ *
+ * @return true if there are no more rows to read, false otherwise.
+ */
+ public boolean nextBatch() throws IOException {
+ Preconditions.checkState(isInitialized, "init() should be called first!");
+
+ // if (rowsRead >= totalRowCount) return false;
+ int batchSize;
+
+ try {
+ batchSize = loadNextBatch();
+ } catch (RuntimeException e) {
+ // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`.
+ throw e;
+ } catch (Throwable e) {
+ throw new IOException(e);
+ }
+
+ if (batchSize == 0) return false;
+
+ long totalDecodeTime = 0, totalLoadTime = 0;
+ for (int i = 0; i < columnReaders.length; i++) {
+ AbstractColumnReader reader = columnReaders[i];
+ long startNs = System.nanoTime();
+ // TODO: read from native reader
+ reader.readBatch(batchSize);
+ // totalDecodeTime += System.nanoTime() - startNs;
+ // startNs = System.nanoTime();
+ vectors[i] = reader.currentBatch();
+ totalLoadTime += System.nanoTime() - startNs;
+ }
+
+ // TODO: (ARROW NATIVE) Add Metrics
+ // SQLMetric decodeMetric = metrics.get("ParquetNativeDecodeTime");
+ // if (decodeMetric != null) {
+ // decodeMetric.add(totalDecodeTime);
+ // }
+ SQLMetric loadMetric = metrics.get("ParquetNativeLoadTime");
+ if (loadMetric != null) {
+ loadMetric.add(totalLoadTime);
+ }
+
+ currentBatch.setNumRows(batchSize);
+ return true;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (columnReaders != null) {
+ for (AbstractColumnReader reader : columnReaders) {
+ if (reader != null) {
+ reader.close();
+ }
+ }
+ }
+ if (importer != null) {
+ importer.close();
+ importer = null;
+ }
+ nativeUtil.close();
+ Native.closeRecordBatchReader(this.handle);
+ }
+
+ @SuppressWarnings("deprecation")
+ private int loadNextBatch() throws Throwable {
+ long startNs = System.nanoTime();
+
+ int batchSize = Native.readNextRecordBatch(this.handle);
+ if (batchSize == 0) {
+ return batchSize;
+ }
+ if (importer != null) importer.close();
+ importer = new CometSchemaImporter(ALLOCATOR);
+
+ List columns = requestedSchema.getColumns();
+ List fields = requestedSchema.getFields();
+ for (int i = 0; i < fields.size(); i++) {
+ // TODO: (ARROW NATIVE) check this. Currently not handling missing columns correctly?
+ if (missingColumns[i]) continue;
+ if (columnReaders[i] != null) columnReaders[i].close();
+ // TODO: (ARROW NATIVE) handle tz, datetime & int96 rebase
+ DataType dataType = sparkSchema.fields()[i].dataType();
+ Type field = fields.get(i);
+ NativeColumnReader reader =
+ new NativeColumnReader(
+ this.handle,
+ i,
+ dataType,
+ field,
+ null,
+ importer,
+ nativeUtil,
+ capacity,
+ useDecimal128,
+ useLegacyDateTimestamp);
+ columnReaders[i] = reader;
+ }
+ return batchSize;
+ }
+
+ // Signature of externalAccums changed from returning a Buffer to returning a Seq. If comet is
+ // expecting a Buffer but the Spark version returns a Seq or vice versa, we get a
+ // method not found exception.
+ @SuppressWarnings("unchecked")
+ private Option> getTaskAccumulator(TaskMetrics taskMetrics) {
+ Method externalAccumsMethod;
+ try {
+ externalAccumsMethod = TaskMetrics.class.getDeclaredMethod("externalAccums");
+ externalAccumsMethod.setAccessible(true);
+ String returnType = externalAccumsMethod.getReturnType().getName();
+ if (returnType.equals("scala.collection.mutable.Buffer")) {
+ return ((Buffer>) externalAccumsMethod.invoke(taskMetrics))
+ .lastOption();
+ } else if (returnType.equals("scala.collection.Seq")) {
+ return ((Seq>) externalAccumsMethod.invoke(taskMetrics)).lastOption();
+ } else {
+ return Option.apply(null); // None
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+ return Option.apply(null); // None
+ }
+ }
+}
diff --git a/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java b/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java
new file mode 100644
index 0000000000..c358999a54
--- /dev/null
+++ b/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java
@@ -0,0 +1,159 @@
+/*
+ * 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.parquet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.arrow.c.ArrowArray;
+import org.apache.arrow.c.ArrowSchema;
+import org.apache.arrow.c.CometSchemaImporter;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.schema.Type;
+import org.apache.spark.sql.types.DataType;
+
+import org.apache.comet.vector.*;
+
+// TODO: extend ColumnReader instead of AbstractColumnReader to reduce code duplication
+public class NativeColumnReader extends AbstractColumnReader {
+ protected static final Logger LOG = LoggerFactory.getLogger(NativeColumnReader.class);
+ protected final BufferAllocator ALLOCATOR = new RootAllocator();
+
+ /**
+ * The current Comet vector holding all the values read by this column reader. Owned by this
+ * reader and MUST be closed after use.
+ */
+ private CometDecodedVector currentVector;
+
+ /** Dictionary values for this column. Only set if the column is using dictionary encoding. */
+ protected CometDictionary dictionary;
+
+ /**
+ * The number of values in the current batch, used when we are skipping importing of Arrow
+ * vectors, in which case we'll simply update the null count of the existing vectors.
+ */
+ int currentNumValues;
+
+ /**
+ * Whether the last loaded vector contains any null value. This is used to determine if we can
+ * skip vector reloading. If the flag is false, Arrow C API will skip to import the validity
+ * buffer, and therefore we cannot skip vector reloading.
+ */
+ boolean hadNull;
+
+ private final CometSchemaImporter importer;
+ private final NativeUtil nativeUtil;
+
+ private ArrowArray array = null;
+ private ArrowSchema schema = null;
+
+ private long nativeBatchHandle = 0xDEADBEEFL;
+ private final int columnNum;
+
+ public NativeColumnReader(
+ long nativeBatchHandle,
+ int columnNum,
+ DataType type,
+ Type fieldType,
+ ColumnDescriptor descriptor,
+ CometSchemaImporter importer,
+ NativeUtil nativeUtil,
+ int batchSize,
+ boolean useDecimal128,
+ boolean useLegacyDateTimestamp) {
+ super(type, fieldType, descriptor, useDecimal128, useLegacyDateTimestamp);
+ assert batchSize > 0 : "Batch size must be positive, found " + batchSize;
+ this.batchSize = batchSize;
+ this.nativeUtil = nativeUtil;
+ this.importer = importer;
+ this.nativeBatchHandle = nativeBatchHandle;
+ this.columnNum = columnNum;
+ initNative();
+ }
+
+ @Override
+ // Override in order to avoid creation of JVM side column readers
+ protected void initNative() {
+ LOG.debug(
+ "Native column reader {} is initialized", String.join(".", this.type.catalogString()));
+ nativeHandle = 0;
+ }
+
+ @Override
+ public void readBatch(int total) {
+ LOG.debug("Reading column batch of size = {}", total);
+
+ this.currentNumValues = total;
+ }
+
+ /** Returns the {@link CometVector} read by this reader. */
+ @Override
+ public CometVector currentBatch() {
+ return loadVector();
+ }
+
+ @Override
+ public void close() {
+ if (currentVector != null) {
+ currentVector.close();
+ currentVector = null;
+ }
+ super.close();
+ }
+
+ /** Returns a decoded {@link CometDecodedVector Comet vector}. */
+ public CometDecodedVector loadVector() {
+
+ LOG.debug("Loading vector for next batch");
+
+ // Close the previous vector first to release struct memory allocated to import Arrow array &
+ // schema from native side, through the C data interface
+ if (currentVector != null) {
+ currentVector.close();
+ }
+
+ // TODO: ARROW NATIVE : Handle Uuid?
+
+ array = ArrowArray.allocateNew(ALLOCATOR);
+ schema = ArrowSchema.allocateNew(ALLOCATOR);
+
+ long arrayAddr = array.memoryAddress();
+ long schemaAddr = schema.memoryAddress();
+
+ Native.currentColumnBatch(nativeBatchHandle, columnNum, arrayAddr, schemaAddr);
+
+ ArrowArray[] arrays = {array};
+ ArrowSchema[] schemas = {schema};
+
+ CometDecodedVector cometVector =
+ (CometDecodedVector)
+ scala.collection.JavaConverters.seqAsJavaList(nativeUtil.importVector(arrays, schemas))
+ .get(0);
+
+ // Update whether the current vector contains any null values. This is used in the following
+ // batch(s) to determine whether we can skip loading the native vector.
+ hadNull = cometVector.hasNull();
+
+ currentVector = cometVector;
+ return currentVector;
+ }
+}
diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java
index 3b0ca35bf9..6be8b28669 100644
--- a/common/src/main/java/org/apache/comet/vector/CometVector.java
+++ b/common/src/main/java/org/apache/comet/vector/CometVector.java
@@ -232,7 +232,7 @@ public DictionaryProvider getDictionaryProvider() {
* @param useDecimal128 Whether to use Decimal128 for decimal column
* @return `CometVector` implementation
*/
- protected static CometVector getVector(
+ public static CometVector getVector(
ValueVector vector, boolean useDecimal128, DictionaryProvider dictionaryProvider) {
if (vector instanceof StructVector) {
return new CometStructVector(vector, useDecimal128, dictionaryProvider);
diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala
index 12e6f971f7..2e64403c64 100644
--- a/common/src/main/scala/org/apache/comet/CometConf.scala
+++ b/common/src/main/scala/org/apache/comet/CometConf.scala
@@ -77,6 +77,27 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(true)
+ val SCAN_NATIVE_COMET = "native_comet"
+ val SCAN_NATIVE_DATAFUSION = "native_datafusion"
+ val SCAN_NATIVE_ICEBERG_COMPAT = "native_iceberg_compat"
+
+ val COMET_NATIVE_SCAN_IMPL: ConfigEntry[String] = conf("spark.comet.scan.impl")
+ .doc(
+ s"The implementation of Comet Native Scan to use. Available modes are '$SCAN_NATIVE_COMET'," +
+ s"'$SCAN_NATIVE_DATAFUSION', and '$SCAN_NATIVE_ICEBERG_COMPAT'. " +
+ s"'$SCAN_NATIVE_COMET' is for the original Comet native scan which uses a jvm based " +
+ "parquet file reader and native column decoding. Supports simple types only " +
+ s"'$SCAN_NATIVE_DATAFUSION' is a fully native implementation of scan based on DataFusion" +
+ s"'$SCAN_NATIVE_ICEBERG_COMPAT' is a native implementation that exposes apis to read " +
+ "parquet columns natively.")
+ .internal()
+ .stringConf
+ .transform(_.toLowerCase(Locale.ROOT))
+ .checkValues(Set(SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT))
+ .createWithDefault(sys.env
+ .getOrElse("COMET_PARQUET_SCAN_IMPL", SCAN_NATIVE_COMET)
+ .toLowerCase(Locale.ROOT))
+
val COMET_PARQUET_PARALLEL_IO_ENABLED: ConfigEntry[Boolean] =
conf("spark.comet.parquet.read.parallel.io.enabled")
.doc(
diff --git a/common/src/main/scala/org/apache/spark/sql/comet/CometArrowUtils.scala b/common/src/main/scala/org/apache/spark/sql/comet/CometArrowUtils.scala
new file mode 100644
index 0000000000..2f4f55fc0b
--- /dev/null
+++ b/common/src/main/scala/org/apache/spark/sql/comet/CometArrowUtils.scala
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet
+
+import scala.collection.JavaConverters._
+
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.complex.MapVector
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, IntervalUnit, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+object CometArrowUtils {
+
+ val rootAllocator = new RootAllocator(Long.MaxValue)
+
+ // todo: support more types.
+
+ /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for TimestampTypes */
+ def toArrowType(dt: DataType, timeZoneId: String): ArrowType = dt match {
+ case BooleanType => ArrowType.Bool.INSTANCE
+ case ByteType => new ArrowType.Int(8, true)
+ case ShortType => new ArrowType.Int(8 * 2, true)
+ case IntegerType => new ArrowType.Int(8 * 4, true)
+ case LongType => new ArrowType.Int(8 * 8, true)
+ case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+ case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+ case StringType => ArrowType.Utf8.INSTANCE
+ case BinaryType => ArrowType.Binary.INSTANCE
+ case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale)
+ case DateType => new ArrowType.Date(DateUnit.DAY)
+ case TimestampType if timeZoneId == null =>
+ throw new IllegalStateException("Missing timezoneId where it is mandatory.")
+ case TimestampType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId)
+ case TimestampNTZType =>
+ new ArrowType.Timestamp(TimeUnit.MICROSECOND, null)
+ case NullType => ArrowType.Null.INSTANCE
+ case _: YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH)
+ case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND)
+ case _ =>
+ throw new IllegalArgumentException()
+ }
+
+ def fromArrowType(dt: ArrowType): DataType = dt match {
+ case ArrowType.Bool.INSTANCE => BooleanType
+ case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 => ByteType
+ case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 2 => ShortType
+ case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 4 => IntegerType
+ case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 8 => LongType
+ case float: ArrowType.FloatingPoint
+ if float.getPrecision() == FloatingPointPrecision.SINGLE =>
+ FloatType
+ case float: ArrowType.FloatingPoint
+ if float.getPrecision() == FloatingPointPrecision.DOUBLE =>
+ DoubleType
+ case ArrowType.Utf8.INSTANCE => StringType
+ case ArrowType.Binary.INSTANCE => BinaryType
+ case d: ArrowType.Decimal => DecimalType(d.getPrecision, d.getScale)
+ case date: ArrowType.Date if date.getUnit == DateUnit.DAY => DateType
+ case ts: ArrowType.Timestamp
+ if ts.getUnit == TimeUnit.MICROSECOND && ts.getTimezone == null =>
+ TimestampNTZType
+ case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType
+ case ArrowType.Null.INSTANCE => NullType
+ case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH =>
+ YearMonthIntervalType()
+ case di: ArrowType.Duration if di.getUnit == TimeUnit.MICROSECOND => DayTimeIntervalType()
+ case _ => throw new IllegalArgumentException()
+ // throw QueryExecutionErrors.unsupportedArrowTypeError(dt)
+ }
+
+ /** Maps field from Spark to Arrow. NOTE: timeZoneId required for TimestampType */
+ def toArrowField(name: String, dt: DataType, nullable: Boolean, timeZoneId: String): Field = {
+ dt match {
+ case ArrayType(elementType, containsNull) =>
+ val fieldType = new FieldType(nullable, ArrowType.List.INSTANCE, null)
+ new Field(
+ name,
+ fieldType,
+ Seq(toArrowField("element", elementType, containsNull, timeZoneId)).asJava)
+ case StructType(fields) =>
+ val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null)
+ new Field(
+ name,
+ fieldType,
+ fields
+ .map { field =>
+ toArrowField(field.name, field.dataType, field.nullable, timeZoneId)
+ }
+ .toSeq
+ .asJava)
+ case MapType(keyType, valueType, valueContainsNull) =>
+ val mapType = new FieldType(nullable, new ArrowType.Map(false), null)
+ // Note: Map Type struct can not be null, Struct Type key field can not be null
+ new Field(
+ name,
+ mapType,
+ Seq(
+ toArrowField(
+ MapVector.DATA_VECTOR_NAME,
+ new StructType()
+ .add(MapVector.KEY_NAME, keyType, nullable = false)
+ .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull),
+ nullable = false,
+ timeZoneId)).asJava)
+ case udt: UserDefinedType[_] => toArrowField(name, udt.sqlType, nullable, timeZoneId)
+ case dataType =>
+ val fieldType = new FieldType(nullable, toArrowType(dataType, timeZoneId), null)
+ new Field(name, fieldType, Seq.empty[Field].asJava)
+ }
+ }
+
+ def fromArrowField(field: Field): DataType = {
+ field.getType match {
+ case _: ArrowType.Map =>
+ val elementField = field.getChildren.get(0)
+ val keyType = fromArrowField(elementField.getChildren.get(0))
+ val valueType = fromArrowField(elementField.getChildren.get(1))
+ MapType(keyType, valueType, elementField.getChildren.get(1).isNullable)
+ case ArrowType.List.INSTANCE =>
+ val elementField = field.getChildren().get(0)
+ val elementType = fromArrowField(elementField)
+ ArrayType(elementType, containsNull = elementField.isNullable)
+ case ArrowType.Struct.INSTANCE =>
+ val fields = field.getChildren().asScala.map { child =>
+ val dt = fromArrowField(child)
+ StructField(child.getName, dt, child.isNullable)
+ }
+ StructType(fields.toArray)
+ case arrowType => fromArrowType(arrowType)
+ }
+ }
+
+ /**
+ * Maps schema from Spark to Arrow. NOTE: timeZoneId required for TimestampType in StructType
+ */
+ def toArrowSchema(schema: StructType, timeZoneId: String): Schema = {
+ new Schema(schema.map { field =>
+ toArrowField(field.name, field.dataType, field.nullable, timeZoneId)
+ }.asJava)
+ }
+
+ def fromArrowSchema(schema: Schema): StructType = {
+ StructType(schema.getFields.asScala.map { field =>
+ val dt = fromArrowField(field)
+ StructField(field.getName, dt, field.isNullable)
+ }.toArray)
+ }
+
+ /** Return Map with conf settings to be used in ArrowPythonRunner */
+ def getPythonRunnerConfMap(conf: SQLConf): Map[String, String] = {
+ val timeZoneConf = Seq(SQLConf.SESSION_LOCAL_TIMEZONE.key -> conf.sessionLocalTimeZone)
+ val pandasColsByName = Seq(
+ SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key ->
+ conf.pandasGroupedMapAssignColumnsByName.toString)
+ val arrowSafeTypeCheck = Seq(
+ SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key ->
+ conf.arrowSafeTypeConversion.toString)
+ Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck: _*)
+ }
+
+}
diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md
index 10459d4d28..94fd9ea7ed 100644
--- a/docs/source/user-guide/compatibility.md
+++ b/docs/source/user-guide/compatibility.md
@@ -133,7 +133,6 @@ The following cast operations are generally compatible with Spark except for the
| string | date | Only supports years between 262143 BC and 262142 AD |
| date | string | |
| timestamp | long | |
-| timestamp | decimal | |
| timestamp | string | |
| timestamp | date | |
diff --git a/native/Cargo.lock b/native/Cargo.lock
index 5a98d74afc..7b00b7bc49 100644
--- a/native/Cargo.lock
+++ b/native/Cargo.lock
@@ -1,6 +1,6 @@
# This file is automatically @generated by Cargo.
# It is not intended for manual editing.
-version = 3
+version = 4
[[package]]
name = "addr2line"
@@ -888,6 +888,7 @@ dependencies = [
"assertables",
"async-trait",
"bytes",
+ "chrono",
"crc32fast",
"criterion",
"datafusion",
@@ -908,6 +909,7 @@ dependencies = [
"lz4_flex",
"mimalloc",
"num",
+ "object_store",
"once_cell",
"parquet",
"paste",
@@ -921,6 +923,7 @@ dependencies = [
"tempfile",
"thiserror",
"tokio",
+ "url",
"zstd 0.11.2+zstd.1.5.2",
]
diff --git a/native/Cargo.toml b/native/Cargo.toml
index 0b39334bd4..72e2386bbb 100644
--- a/native/Cargo.toml
+++ b/native/Cargo.toml
@@ -57,6 +57,8 @@ num = "0.4"
rand = "0.8"
regex = "1.9.6"
thiserror = "1"
+object_store = "0.11.0"
+url = "2.2"
[profile.release]
debug = true
diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml
index 8937236dda..88c81be313 100644
--- a/native/core/Cargo.toml
+++ b/native/core/Cargo.toml
@@ -62,7 +62,7 @@ bytes = "1.5.0"
tempfile = "3.8.0"
itertools = "0.11.0"
paste = "1.0.14"
-datafusion-common = { workspace = true }
+datafusion-common = { workspace = true, features= ["object_store"] }
datafusion = { workspace = true }
datafusion-functions-nested = { workspace = true }
datafusion-expr = { workspace = true }
@@ -74,6 +74,9 @@ crc32fast = "1.3.2"
simd-adler32 = "0.3.7"
datafusion-comet-spark-expr = { workspace = true }
datafusion-comet-proto = { workspace = true }
+object_store = { workspace = true }
+url = { workspace = true }
+chrono = { workspace = true }
[dev-dependencies]
pprof = { version = "0.13.0", features = ["flamegraph"] }
diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs
index b3c33b7948..fe29d8da14 100644
--- a/native/core/src/execution/jni_api.rs
+++ b/native/core/src/execution/jni_api.rs
@@ -77,6 +77,8 @@ struct ExecutionContext {
pub task_attempt_id: i64,
/// The deserialized Spark plan
pub spark_plan: Operator,
+ /// The number of partitions
+ pub partition_count: usize,
/// The DataFusion root operator converted from the `spark_plan`
pub root_op: Option>,
/// The input sources for the DataFusion plan
@@ -156,6 +158,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
id: jlong,
iterators: jobjectArray,
serialized_query: jbyteArray,
+ partition_count: jint,
metrics_node: JObject,
comet_task_memory_manager_obj: JObject,
batch_size: jint,
@@ -223,6 +226,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
id,
task_attempt_id,
spark_plan,
+ partition_count: partition_count as usize,
root_op: None,
scans: vec![],
input_sources,
@@ -472,6 +476,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
let (scans, root_op) = planner.create_plan(
&exec_context.spark_plan,
&mut exec_context.input_sources.clone(),
+ exec_context.partition_count,
)?;
let physical_plan_time = start.elapsed();
@@ -491,7 +496,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
.as_ref()
.unwrap()
.native_plan
- .execute(0, task_ctx)?;
+ .execute(partition as usize, task_ctx)?;
exec_context.stream = Some(stream);
} else {
// Pull input batches
diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs
index 27a0ad58e7..533706f4e2 100644
--- a/native/core/src/execution/planner.rs
+++ b/native/core/src/execution/planner.rs
@@ -71,6 +71,11 @@ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctio
use crate::execution::shuffle::CompressionCodec;
use crate::execution::spark_plan::SparkPlan;
+use crate::parquet::parquet_support::SparkParquetOptions;
+use crate::parquet::schema_adapter::SparkSchemaAdapterFactory;
+use datafusion::datasource::listing::PartitionedFile;
+use datafusion::datasource::physical_plan::parquet::ParquetExecBuilder;
+use datafusion::datasource::physical_plan::FileScanConfig;
use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec;
use datafusion_comet_proto::{
spark_expression::{
@@ -91,11 +96,13 @@ use datafusion_comet_spark_expr::{
SparkCastOptions, StartsWith, Stddev, StringSpaceExpr, SubstringExpr, SumDecimal,
TimestampTruncExpr, ToJson, UnboundColumn, Variance,
};
+use datafusion_common::config::TableParquetOptions;
use datafusion_common::scalar::ScalarStructBuilder;
use datafusion_common::{
tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter},
JoinType as DFJoinType, ScalarValue,
};
+use datafusion_execution::object_store::ObjectStoreUrl;
use datafusion_expr::{
AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits,
WindowFunctionDefinition,
@@ -107,8 +114,10 @@ use datafusion_physical_expr::LexOrdering;
use itertools::Itertools;
use jni::objects::GlobalRef;
use num::{BigInt, ToPrimitive};
+use object_store::path::Path;
use std::cmp::max;
use std::{collections::HashMap, sync::Arc};
+use url::Url;
// For clippy error on type_complexity.
type PhyAggResult = Result, ExecutionError>;
@@ -897,12 +906,13 @@ impl PhysicalPlanner {
&'a self,
spark_plan: &'a Operator,
inputs: &mut Vec>,
+ partition_count: usize,
) -> Result<(Vec, Arc), ExecutionError> {
let children = &spark_plan.children;
match spark_plan.op_struct.as_ref().unwrap() {
OpStruct::Projection(project) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let exprs: PhyExprResult = project
.project_list
.iter()
@@ -923,7 +933,7 @@ impl PhysicalPlanner {
}
OpStruct::Filter(filter) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let predicate =
self.create_expr(filter.predicate.as_ref().unwrap(), child.schema())?;
@@ -938,7 +948,7 @@ impl PhysicalPlanner {
}
OpStruct::HashAgg(agg) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let group_exprs: PhyExprResult = agg
.grouping_exprs
@@ -1017,7 +1027,7 @@ impl PhysicalPlanner {
OpStruct::Limit(limit) => {
assert!(children.len() == 1);
let num = limit.limit;
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let limit = Arc::new(LocalLimitExec::new(
Arc::clone(&child.native_plan),
@@ -1030,7 +1040,7 @@ impl PhysicalPlanner {
}
OpStruct::Sort(sort) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let exprs: Result, ExecutionError> = sort
.sort_orders
@@ -1060,6 +1070,148 @@ impl PhysicalPlanner {
)),
))
}
+ OpStruct::NativeScan(scan) => {
+ let data_schema = convert_spark_types_to_arrow_schema(scan.data_schema.as_slice());
+ let required_schema: SchemaRef =
+ convert_spark_types_to_arrow_schema(scan.required_schema.as_slice());
+ let partition_schema: SchemaRef =
+ convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice());
+ let projection_vector: Vec = scan
+ .projection_vector
+ .iter()
+ .map(|offset| *offset as usize)
+ .collect();
+
+ // Convert the Spark expressions to Physical expressions
+ let data_filters: Result>, ExecutionError> = scan
+ .data_filters
+ .iter()
+ .map(|expr| self.create_expr(expr, Arc::clone(&required_schema)))
+ .collect();
+
+ // Create a conjunctive form of the vector because ParquetExecBuilder takes
+ // a single expression
+ let data_filters = data_filters?;
+ let cnf_data_filters = data_filters.clone().into_iter().reduce(|left, right| {
+ Arc::new(BinaryExpr::new(
+ left,
+ datafusion::logical_expr::Operator::And,
+ right,
+ ))
+ });
+
+ let object_store = object_store::local::LocalFileSystem::new();
+ // register the object store with the runtime environment
+ let url = Url::try_from("file://").unwrap();
+ self.session_ctx
+ .runtime_env()
+ .register_object_store(&url, Arc::new(object_store));
+
+ // Generate file groups
+ let mut file_groups: Vec> =
+ Vec::with_capacity(partition_count);
+ scan.file_partitions.iter().try_for_each(|partition| {
+ let mut files = Vec::with_capacity(partition.partitioned_file.len());
+ partition.partitioned_file.iter().try_for_each(|file| {
+ assert!(file.start + file.length <= file.file_size);
+
+ let mut partitioned_file = PartitionedFile::new_with_range(
+ String::new(), // Dummy file path.
+ file.file_size as u64,
+ file.start,
+ file.start + file.length,
+ );
+
+ // Spark sends the path over as URL-encoded, parse that first.
+ let url = Url::parse(file.file_path.as_ref()).unwrap();
+ // Convert that to a Path object to use in the PartitionedFile.
+ let path = Path::from_url_path(url.path()).unwrap();
+ partitioned_file.object_meta.location = path;
+
+ // Process partition values
+ // Create an empty input schema for partition values because they are all literals.
+ let empty_schema = Arc::new(Schema::empty());
+ let partition_values: Result, _> = file
+ .partition_values
+ .iter()
+ .map(|partition_value| {
+ let literal = self.create_expr(
+ partition_value,
+ Arc::::clone(&empty_schema),
+ )?;
+ literal
+ .as_any()
+ .downcast_ref::()
+ .ok_or_else(|| {
+ ExecutionError::GeneralError(
+ "Expected literal of partition value".to_string(),
+ )
+ })
+ .map(|literal| literal.value().clone())
+ })
+ .collect();
+ let partition_values = partition_values?;
+
+ partitioned_file.partition_values = partition_values;
+
+ files.push(partitioned_file);
+ Ok::<(), ExecutionError>(())
+ })?;
+
+ file_groups.push(files);
+ Ok::<(), ExecutionError>(())
+ })?;
+
+ // TODO: I think we can remove partition_count in the future, but leave for testing.
+ assert_eq!(file_groups.len(), partition_count);
+
+ let object_store_url = ObjectStoreUrl::local_filesystem();
+ let partition_fields: Vec = partition_schema
+ .fields()
+ .iter()
+ .map(|field| {
+ Field::new(field.name(), field.data_type().clone(), field.is_nullable())
+ })
+ .collect_vec();
+ let mut file_scan_config =
+ FileScanConfig::new(object_store_url, Arc::clone(&data_schema))
+ .with_file_groups(file_groups)
+ .with_table_partition_cols(partition_fields);
+
+ assert_eq!(
+ projection_vector.len(),
+ required_schema.fields.len() + partition_schema.fields.len()
+ );
+ file_scan_config = file_scan_config.with_projection(Some(projection_vector));
+
+ let mut table_parquet_options = TableParquetOptions::new();
+ // TODO: Maybe these are configs?
+ table_parquet_options.global.pushdown_filters = true;
+ table_parquet_options.global.reorder_filters = true;
+
+ let mut spark_parquet_options = SparkParquetOptions::new(
+ EvalMode::Legacy,
+ scan.session_timezone.as_str(),
+ false,
+ );
+ spark_parquet_options.allow_cast_unsigned_ints = true;
+
+ let mut builder = ParquetExecBuilder::new(file_scan_config)
+ .with_table_parquet_options(table_parquet_options)
+ .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new(
+ spark_parquet_options,
+ )));
+
+ if let Some(filter) = cnf_data_filters {
+ builder = builder.with_predicate(filter);
+ }
+
+ let scan = builder.build();
+ Ok((
+ vec![],
+ Arc::new(SparkPlan::new(spark_plan.plan_id, Arc::new(scan), vec![])),
+ ))
+ }
OpStruct::Scan(scan) => {
let data_types = scan.fields.iter().map(to_arrow_datatype).collect_vec();
@@ -1090,7 +1242,7 @@ impl PhysicalPlanner {
}
OpStruct::ShuffleWriter(writer) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let partitioning = self
.create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?;
@@ -1128,7 +1280,7 @@ impl PhysicalPlanner {
}
OpStruct::Expand(expand) => {
assert!(children.len() == 1);
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let mut projections = vec![];
let mut projection = vec![];
@@ -1189,6 +1341,7 @@ impl PhysicalPlanner {
&join.right_join_keys,
join.join_type,
&join.condition,
+ partition_count,
)?;
let sort_options = join
@@ -1262,6 +1415,7 @@ impl PhysicalPlanner {
&join.right_join_keys,
join.join_type,
&join.condition,
+ partition_count,
)?;
// HashJoinExec may cache the input batch internally. We need
@@ -1316,7 +1470,7 @@ impl PhysicalPlanner {
}
}
OpStruct::Window(wnd) => {
- let (scans, child) = self.create_plan(&children[0], inputs)?;
+ let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;
let input_schema = child.schema();
let sort_exprs: Result, ExecutionError> = wnd
.order_by_list
@@ -1369,10 +1523,11 @@ impl PhysicalPlanner {
right_join_keys: &[Expr],
join_type: i32,
condition: &Option,
+ partition_count: usize,
) -> Result<(JoinParameters, Vec), ExecutionError> {
assert!(children.len() == 2);
- let (mut left_scans, left) = self.create_plan(&children[0], inputs)?;
- let (mut right_scans, right) = self.create_plan(&children[1], inputs)?;
+ let (mut left_scans, left) = self.create_plan(&children[0], inputs, partition_count)?;
+ let (mut right_scans, right) = self.create_plan(&children[1], inputs, partition_count)?;
left_scans.append(&mut right_scans);
@@ -2325,6 +2480,23 @@ fn from_protobuf_eval_mode(value: i32) -> Result {
}
}
+fn convert_spark_types_to_arrow_schema(
+ spark_types: &[spark_operator::SparkStructField],
+) -> SchemaRef {
+ let arrow_fields = spark_types
+ .iter()
+ .map(|spark_type| {
+ Field::new(
+ String::clone(&spark_type.name),
+ to_arrow_datatype(spark_type.data_type.as_ref().unwrap()),
+ spark_type.nullable,
+ )
+ })
+ .collect_vec();
+ let arrow_schema: SchemaRef = Arc::new(Schema::new(arrow_fields));
+ arrow_schema
+}
+
#[cfg(test)]
mod tests {
use std::{sync::Arc, task::Poll};
@@ -2371,7 +2543,7 @@ mod tests {
let input_array = DictionaryArray::new(keys, Arc::new(values));
let input_batch = InputBatch::Batch(vec![Arc::new(input_array)], row_count);
- let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![]).unwrap();
+ let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![], 1).unwrap();
scans[0].set_input_batch(input_batch);
let session_ctx = SessionContext::new();
@@ -2453,7 +2625,7 @@ mod tests {
let input_array = DictionaryArray::new(keys, Arc::new(values));
let input_batch = InputBatch::Batch(vec![Arc::new(input_array)], row_count);
- let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![]).unwrap();
+ let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![], 1).unwrap();
// Scan's schema is determined by the input batch, so we need to set it before execution.
scans[0].set_input_batch(input_batch);
@@ -2513,7 +2685,7 @@ mod tests {
let op = create_filter(op_scan, 0);
let planner = PhysicalPlanner::default();
- let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![]).unwrap();
+ let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut vec![], 1).unwrap();
let scan = &mut scans[0];
scan.set_input_batch(InputBatch::EOF);
@@ -2592,7 +2764,7 @@ mod tests {
let op = create_filter(op_scan, 0);
let planner = PhysicalPlanner::default();
- let (mut _scans, filter_exec) = planner.create_plan(&op, &mut vec![]).unwrap();
+ let (mut _scans, filter_exec) = planner.create_plan(&op, &mut vec![], 1).unwrap();
assert_eq!("FilterExec", filter_exec.native_plan.name());
assert_eq!(1, filter_exec.children.len());
@@ -2616,7 +2788,7 @@ mod tests {
let planner = PhysicalPlanner::default();
- let (_scans, hash_join_exec) = planner.create_plan(&op_join, &mut vec![]).unwrap();
+ let (_scans, hash_join_exec) = planner.create_plan(&op_join, &mut vec![], 1).unwrap();
assert_eq!("HashJoinExec", hash_join_exec.native_plan.name());
assert_eq!(2, hash_join_exec.children.len());
diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs
index 6bea31f441..b0a74864cf 100644
--- a/native/core/src/parquet/mod.rs
+++ b/native/core/src/parquet/mod.rs
@@ -21,8 +21,11 @@ pub use mutable_vector::*;
#[macro_use]
pub mod util;
+pub mod parquet_support;
pub mod read;
+pub mod schema_adapter;
+use std::task::Poll;
use std::{boxed::Box, ptr::NonNull, sync::Arc};
use crate::errors::{try_unwrap_or_throw, CometError};
@@ -39,14 +42,26 @@ use jni::{
},
};
+use self::util::jni::TypePromotionInfo;
+use crate::execution::operators::ExecutionError;
use crate::execution::utils::SparkArrowConvert;
+use crate::parquet::data_type::AsBytes;
+use crate::parquet::parquet_support::SparkParquetOptions;
+use crate::parquet::schema_adapter::SparkSchemaAdapterFactory;
use arrow::buffer::{Buffer, MutableBuffer};
-use jni::objects::{JBooleanArray, JLongArray, JPrimitiveArray, ReleaseMode};
+use arrow_array::{Array, RecordBatch};
+use datafusion::datasource::listing::PartitionedFile;
+use datafusion::datasource::physical_plan::parquet::ParquetExecBuilder;
+use datafusion::datasource::physical_plan::FileScanConfig;
+use datafusion::physical_plan::ExecutionPlan;
+use datafusion_comet_spark_expr::EvalMode;
+use datafusion_common::config::TableParquetOptions;
+use datafusion_execution::{SendableRecordBatchStream, TaskContext};
+use futures::{poll, StreamExt};
+use jni::objects::{JBooleanArray, JByteArray, JLongArray, JPrimitiveArray, JString, ReleaseMode};
+use jni::sys::jstring;
use read::ColumnReader;
-use util::jni::{convert_column_descriptor, convert_encoding};
-
-use self::util::jni::TypePromotionInfo;
-
+use util::jni::{convert_column_descriptor, convert_encoding, deserialize_schema, get_file_path};
/// Parquet read context maintained across multiple JNI calls.
struct Context {
pub column_reader: ColumnReader,
@@ -580,3 +595,205 @@ fn from_u8_slice(src: &mut [u8]) -> &mut [i8] {
let raw_ptr = src.as_mut_ptr() as *mut i8;
unsafe { std::slice::from_raw_parts_mut(raw_ptr, src.len()) }
}
+
+// TODO: (ARROW NATIVE) remove this if not needed.
+enum ParquetReaderState {
+ Init,
+ Reading,
+ Complete,
+}
+/// Parquet read context maintained across multiple JNI calls.
+struct BatchContext {
+ runtime: tokio::runtime::Runtime,
+ batch_stream: Option,
+ current_batch: Option,
+ reader_state: ParquetReaderState,
+}
+
+#[inline]
+fn get_batch_context<'a>(handle: jlong) -> Result<&'a mut BatchContext, CometError> {
+ unsafe {
+ (handle as *mut BatchContext)
+ .as_mut()
+ .ok_or_else(|| CometError::NullPointer("null batch context handle".to_string()))
+ }
+}
+
+/*
+#[inline]
+fn get_batch_reader<'a>(handle: jlong) -> Result<&'a mut ParquetRecordBatchReader, CometError> {
+ Ok(&mut get_batch_context(handle)?.batch_reader.unwrap())
+}
+*/
+
+/// # Safety
+/// This function is inherently unsafe since it deals with raw pointers passed from JNI.
+#[no_mangle]
+pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBatchReader(
+ e: JNIEnv,
+ _jclass: JClass,
+ file_path: jstring,
+ file_size: jlong,
+ start: jlong,
+ length: jlong,
+ required_schema: jbyteArray,
+ session_timezone: jstring,
+) -> jlong {
+ try_unwrap_or_throw(&e, |mut env| unsafe {
+ let path: String = env
+ .get_string(&JString::from_raw(file_path))
+ .unwrap()
+ .into();
+ let batch_stream: Option;
+ // TODO: (ARROW NATIVE) Use the common global runtime
+ let runtime = tokio::runtime::Builder::new_multi_thread()
+ .enable_all()
+ .build()?;
+
+ // EXPERIMENTAL - BEGIN
+ //TODO: Need an execution context and a spark plan equivalent so that we can reuse
+ // code from jni_api.rs
+ let (object_store_url, object_store_path) = get_file_path(path.clone()).unwrap();
+ // TODO: (ARROW NATIVE) - Remove code duplication between this and POC 1
+ // copy the input on-heap buffer to native
+ let required_schema_array = JByteArray::from_raw(required_schema);
+ let required_schema_buffer = env.convert_byte_array(&required_schema_array)?;
+ let required_schema_arrow = deserialize_schema(required_schema_buffer.as_bytes())?;
+ let mut partitioned_file = PartitionedFile::new_with_range(
+ String::new(), // Dummy file path. We will override this with our path so that url encoding does not occur
+ file_size as u64,
+ start,
+ start + length,
+ );
+ partitioned_file.object_meta.location = object_store_path;
+ // We build the file scan config with the *required* schema so that the reader knows
+ // the output schema we want
+ let file_scan_config = FileScanConfig::new(object_store_url, Arc::new(required_schema_arrow))
+ .with_file(partitioned_file)
+ // TODO: (ARROW NATIVE) - do partition columns in native
+ // - will need partition schema and partition values to do so
+ // .with_table_partition_cols(partition_fields)
+ ;
+ let mut table_parquet_options = TableParquetOptions::new();
+ // TODO: Maybe these are configs?
+ table_parquet_options.global.pushdown_filters = true;
+ table_parquet_options.global.reorder_filters = true;
+ let session_timezone: String = env
+ .get_string(&JString::from_raw(session_timezone))
+ .unwrap()
+ .into();
+
+ let mut spark_parquet_options =
+ SparkParquetOptions::new(EvalMode::Legacy, session_timezone.as_str(), false);
+ spark_parquet_options.allow_cast_unsigned_ints = true;
+
+ let builder2 = ParquetExecBuilder::new(file_scan_config)
+ .with_table_parquet_options(table_parquet_options)
+ .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new(
+ spark_parquet_options,
+ )));
+
+ //TODO: (ARROW NATIVE) - predicate pushdown??
+ // builder = builder.with_predicate(filter);
+
+ let scan = builder2.build();
+ let ctx = TaskContext::default();
+ let partition_index: usize = 0;
+ batch_stream = Some(scan.execute(partition_index, Arc::new(ctx))?);
+
+ // EXPERIMENTAL - END
+
+ let ctx = BatchContext {
+ runtime,
+ batch_stream,
+ current_batch: None,
+ reader_state: ParquetReaderState::Init,
+ };
+ let res = Box::new(ctx);
+ Ok(Box::into_raw(res) as i64)
+ })
+}
+
+#[no_mangle]
+pub extern "system" fn Java_org_apache_comet_parquet_Native_readNextRecordBatch(
+ e: JNIEnv,
+ _jclass: JClass,
+ handle: jlong,
+) -> jint {
+ try_unwrap_or_throw(&e, |_env| {
+ let context = get_batch_context(handle)?;
+ let mut rows_read: i32 = 0;
+ let batch_stream = context.batch_stream.as_mut().unwrap();
+ let runtime = &context.runtime;
+
+ loop {
+ let next_item = batch_stream.next();
+ let poll_batch: Poll>> =
+ runtime.block_on(async { poll!(next_item) });
+
+ match poll_batch {
+ Poll::Ready(Some(batch)) => {
+ let batch = batch?;
+ rows_read = batch.num_rows() as i32;
+ context.current_batch = Some(batch);
+ context.reader_state = ParquetReaderState::Reading;
+ break;
+ }
+ Poll::Ready(None) => {
+ // EOF
+
+ // TODO: (ARROW NATIVE) We can update metrics here
+ // crate::execution::jni_api::update_metrics(&mut env, exec_context)?;
+
+ context.current_batch = None;
+ context.reader_state = ParquetReaderState::Complete;
+ break;
+ }
+ Poll::Pending => {
+ // TODO: (ARROW NATIVE): Just keeping polling??
+ // Ideally we want to yield to avoid consuming CPU while blocked on IO ??
+ continue;
+ }
+ }
+ }
+ Ok(rows_read)
+ })
+}
+
+#[no_mangle]
+pub extern "system" fn Java_org_apache_comet_parquet_Native_currentColumnBatch(
+ e: JNIEnv,
+ _jclass: JClass,
+ handle: jlong,
+ column_idx: jint,
+ array_addr: jlong,
+ schema_addr: jlong,
+) {
+ try_unwrap_or_throw(&e, |_env| {
+ let context = get_batch_context(handle)?;
+ let batch_reader = context
+ .current_batch
+ .as_mut()
+ .ok_or_else(|| CometError::Execution {
+ source: ExecutionError::GeneralError("There is no more data to read".to_string()),
+ });
+ let data = batch_reader?.column(column_idx as usize).into_data();
+ data.move_to_spark(array_addr, schema_addr)
+ .map_err(|e| e.into())
+ })
+}
+
+#[no_mangle]
+pub extern "system" fn Java_org_apache_comet_parquet_Native_closeRecordBatchReader(
+ env: JNIEnv,
+ _jclass: JClass,
+ handle: jlong,
+) {
+ try_unwrap_or_throw(&env, |_| {
+ unsafe {
+ let ctx = handle as *mut BatchContext;
+ let _ = Box::from_raw(ctx);
+ };
+ Ok(())
+ })
+}
diff --git a/native/core/src/parquet/parquet_support.rs b/native/core/src/parquet/parquet_support.rs
new file mode 100644
index 0000000000..248f2babd6
--- /dev/null
+++ b/native/core/src/parquet/parquet_support.rs
@@ -0,0 +1,2338 @@
+// 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.
+
+use arrow::{
+ array::{
+ cast::AsArray,
+ types::{Date32Type, Int16Type, Int32Type, Int8Type},
+ Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+ GenericStringArray, Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait,
+ PrimitiveArray,
+ },
+ compute::{cast_with_options, take, unary, CastOptions},
+ datatypes::{
+ ArrowPrimitiveType, Decimal128Type, DecimalType, Float32Type, Float64Type, Int64Type,
+ TimestampMicrosecondType,
+ },
+ error::ArrowError,
+ util::display::FormatOptions,
+};
+use arrow_array::builder::StringBuilder;
+use arrow_array::{DictionaryArray, StringArray, StructArray};
+use arrow_schema::DataType;
+use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike};
+use datafusion_comet_spark_expr::utils::array_with_timezone;
+use datafusion_comet_spark_expr::{timezone, EvalMode, SparkError, SparkResult};
+use datafusion_common::{cast::as_generic_string_array, Result as DataFusionResult, ScalarValue};
+use datafusion_expr::ColumnarValue;
+// use datafusion_physical_expr::PhysicalExpr;
+use num::{
+ cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num,
+ ToPrimitive,
+};
+use regex::Regex;
+use std::collections::HashMap;
+use std::str::FromStr;
+use std::{fmt::Debug, hash::Hash, num::Wrapping, sync::Arc};
+
+static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f");
+
+const MICROS_PER_SECOND: i64 = 1000000;
+
+static PARQUET_OPTIONS: CastOptions = CastOptions {
+ safe: true,
+ format_options: FormatOptions::new()
+ .with_timestamp_tz_format(TIMESTAMP_FORMAT)
+ .with_timestamp_format(TIMESTAMP_FORMAT),
+};
+
+struct TimeStampInfo {
+ year: i32,
+ month: u32,
+ day: u32,
+ hour: u32,
+ minute: u32,
+ second: u32,
+ microsecond: u32,
+}
+
+impl Default for TimeStampInfo {
+ fn default() -> Self {
+ TimeStampInfo {
+ year: 1,
+ month: 1,
+ day: 1,
+ hour: 0,
+ minute: 0,
+ second: 0,
+ microsecond: 0,
+ }
+ }
+}
+
+impl TimeStampInfo {
+ pub fn with_year(&mut self, year: i32) -> &mut Self {
+ self.year = year;
+ self
+ }
+
+ pub fn with_month(&mut self, month: u32) -> &mut Self {
+ self.month = month;
+ self
+ }
+
+ pub fn with_day(&mut self, day: u32) -> &mut Self {
+ self.day = day;
+ self
+ }
+
+ pub fn with_hour(&mut self, hour: u32) -> &mut Self {
+ self.hour = hour;
+ self
+ }
+
+ pub fn with_minute(&mut self, minute: u32) -> &mut Self {
+ self.minute = minute;
+ self
+ }
+
+ pub fn with_second(&mut self, second: u32) -> &mut Self {
+ self.second = second;
+ self
+ }
+
+ pub fn with_microsecond(&mut self, microsecond: u32) -> &mut Self {
+ self.microsecond = microsecond;
+ self
+ }
+}
+
+macro_rules! cast_utf8_to_int {
+ ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{
+ let len = $array.len();
+ let mut cast_array = PrimitiveArray::<$array_type>::builder(len);
+ for i in 0..len {
+ if $array.is_null(i) {
+ cast_array.append_null()
+ } else if let Some(cast_value) = $cast_method($array.value(i), $eval_mode)? {
+ cast_array.append_value(cast_value);
+ } else {
+ cast_array.append_null()
+ }
+ }
+ let result: SparkResult = Ok(Arc::new(cast_array.finish()) as ArrayRef);
+ result
+ }};
+}
+macro_rules! cast_utf8_to_timestamp {
+ ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident, $tz:expr) => {{
+ let len = $array.len();
+ let mut cast_array = PrimitiveArray::<$array_type>::builder(len).with_timezone("UTC");
+ for i in 0..len {
+ if $array.is_null(i) {
+ cast_array.append_null()
+ } else if let Ok(Some(cast_value)) =
+ $cast_method($array.value(i).trim(), $eval_mode, $tz)
+ {
+ cast_array.append_value(cast_value);
+ } else {
+ cast_array.append_null()
+ }
+ }
+ let result: ArrayRef = Arc::new(cast_array.finish()) as ArrayRef;
+ result
+ }};
+}
+
+macro_rules! cast_float_to_string {
+ ($from:expr, $eval_mode:expr, $type:ty, $output_type:ty, $offset_type:ty) => {{
+
+ fn cast(
+ from: &dyn Array,
+ _eval_mode: EvalMode,
+ ) -> SparkResult
+ where
+ OffsetSize: OffsetSizeTrait, {
+ let array = from.as_any().downcast_ref::<$output_type>().unwrap();
+
+ // If the absolute number is less than 10,000,000 and greater or equal than 0.001, the
+ // result is expressed without scientific notation with at least one digit on either side of
+ // the decimal point. Otherwise, Spark uses a mantissa followed by E and an
+ // exponent. The mantissa has an optional leading minus sign followed by one digit to the
+ // left of the decimal point, and the minimal number of digits greater than zero to the
+ // right. The exponent has and optional leading minus sign.
+ // source: https://docs.databricks.com/en/sql/language-manual/functions/cast.html
+
+ const LOWER_SCIENTIFIC_BOUND: $type = 0.001;
+ const UPPER_SCIENTIFIC_BOUND: $type = 10000000.0;
+
+ let output_array = array
+ .iter()
+ .map(|value| match value {
+ Some(value) if value == <$type>::INFINITY => Ok(Some("Infinity".to_string())),
+ Some(value) if value == <$type>::NEG_INFINITY => Ok(Some("-Infinity".to_string())),
+ Some(value)
+ if (value.abs() < UPPER_SCIENTIFIC_BOUND
+ && value.abs() >= LOWER_SCIENTIFIC_BOUND)
+ || value.abs() == 0.0 =>
+ {
+ let trailing_zero = if value.fract() == 0.0 { ".0" } else { "" };
+
+ Ok(Some(format!("{value}{trailing_zero}")))
+ }
+ Some(value)
+ if value.abs() >= UPPER_SCIENTIFIC_BOUND
+ || value.abs() < LOWER_SCIENTIFIC_BOUND =>
+ {
+ let formatted = format!("{value:E}");
+
+ if formatted.contains(".") {
+ Ok(Some(formatted))
+ } else {
+ // `formatted` is already in scientific notation and can be split up by E
+ // in order to add the missing trailing 0 which gets removed for numbers with a fraction of 0.0
+ let prepare_number: Vec<&str> = formatted.split("E").collect();
+
+ let coefficient = prepare_number[0];
+
+ let exponent = prepare_number[1];
+
+ Ok(Some(format!("{coefficient}.0E{exponent}")))
+ }
+ }
+ Some(value) => Ok(Some(value.to_string())),
+ _ => Ok(None),
+ })
+ .collect::, SparkError>>()?;
+
+ Ok(Arc::new(output_array))
+ }
+
+ cast::<$offset_type>($from, $eval_mode)
+ }};
+}
+
+macro_rules! cast_int_to_int_macro {
+ (
+ $array: expr,
+ $eval_mode:expr,
+ $from_arrow_primitive_type: ty,
+ $to_arrow_primitive_type: ty,
+ $from_data_type: expr,
+ $to_native_type: ty,
+ $spark_from_data_type_name: expr,
+ $spark_to_data_type_name: expr
+ ) => {{
+ let cast_array = $array
+ .as_any()
+ .downcast_ref::>()
+ .unwrap();
+ let spark_int_literal_suffix = match $from_data_type {
+ &DataType::Int64 => "L",
+ &DataType::Int16 => "S",
+ &DataType::Int8 => "T",
+ _ => "",
+ };
+
+ let output_array = match $eval_mode {
+ EvalMode::Legacy => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ Ok::, SparkError>(Some(value as $to_native_type))
+ }
+ _ => Ok(None),
+ })
+ .collect::, _>>(),
+ _ => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let res = <$to_native_type>::try_from(value);
+ if res.is_err() {
+ Err(cast_overflow(
+ &(value.to_string() + spark_int_literal_suffix),
+ $spark_from_data_type_name,
+ $spark_to_data_type_name,
+ ))
+ } else {
+ Ok::, SparkError>(Some(res.unwrap()))
+ }
+ }
+ _ => Ok(None),
+ })
+ .collect::, _>>(),
+ }?;
+ let result: SparkResult = Ok(Arc::new(output_array) as ArrayRef);
+ result
+ }};
+}
+
+// When Spark casts to Byte/Short Types, it does not cast directly to Byte/Short.
+// It casts to Int first and then to Byte/Short. Because of potential overflows in the Int cast,
+// this can cause unexpected Short/Byte cast results. Replicate this behavior.
+macro_rules! cast_float_to_int16_down {
+ (
+ $array:expr,
+ $eval_mode:expr,
+ $src_array_type:ty,
+ $dest_array_type:ty,
+ $rust_src_type:ty,
+ $rust_dest_type:ty,
+ $src_type_str:expr,
+ $dest_type_str:expr,
+ $format_str:expr
+ ) => {{
+ let cast_array = $array
+ .as_any()
+ .downcast_ref::<$src_array_type>()
+ .expect(concat!("Expected a ", stringify!($src_array_type)));
+
+ let output_array = match $eval_mode {
+ EvalMode::Ansi => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let is_overflow = value.is_nan() || value.abs() as i32 == i32::MAX;
+ if is_overflow {
+ return Err(cast_overflow(
+ &format!($format_str, value).replace("e", "E"),
+ $src_type_str,
+ $dest_type_str,
+ ));
+ }
+ let i32_value = value as i32;
+ <$rust_dest_type>::try_from(i32_value)
+ .map_err(|_| {
+ cast_overflow(
+ &format!($format_str, value).replace("e", "E"),
+ $src_type_str,
+ $dest_type_str,
+ )
+ })
+ .map(Some)
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ _ => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let i32_value = value as i32;
+ Ok::, SparkError>(Some(
+ i32_value as $rust_dest_type,
+ ))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ };
+ Ok(Arc::new(output_array) as ArrayRef)
+ }};
+}
+
+macro_rules! cast_float_to_int32_up {
+ (
+ $array:expr,
+ $eval_mode:expr,
+ $src_array_type:ty,
+ $dest_array_type:ty,
+ $rust_src_type:ty,
+ $rust_dest_type:ty,
+ $src_type_str:expr,
+ $dest_type_str:expr,
+ $max_dest_val:expr,
+ $format_str:expr
+ ) => {{
+ let cast_array = $array
+ .as_any()
+ .downcast_ref::<$src_array_type>()
+ .expect(concat!("Expected a ", stringify!($src_array_type)));
+
+ let output_array = match $eval_mode {
+ EvalMode::Ansi => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let is_overflow =
+ value.is_nan() || value.abs() as $rust_dest_type == $max_dest_val;
+ if is_overflow {
+ return Err(cast_overflow(
+ &format!($format_str, value).replace("e", "E"),
+ $src_type_str,
+ $dest_type_str,
+ ));
+ }
+ Ok(Some(value as $rust_dest_type))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ _ => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ Ok::, SparkError>(Some(value as $rust_dest_type))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ };
+ Ok(Arc::new(output_array) as ArrayRef)
+ }};
+}
+
+// When Spark casts to Byte/Short Types, it does not cast directly to Byte/Short.
+// It casts to Int first and then to Byte/Short. Because of potential overflows in the Int cast,
+// this can cause unexpected Short/Byte cast results. Replicate this behavior.
+macro_rules! cast_decimal_to_int16_down {
+ (
+ $array:expr,
+ $eval_mode:expr,
+ $dest_array_type:ty,
+ $rust_dest_type:ty,
+ $dest_type_str:expr,
+ $precision:expr,
+ $scale:expr
+ ) => {{
+ let cast_array = $array
+ .as_any()
+ .downcast_ref::()
+ .expect(concat!("Expected a Decimal128ArrayType"));
+
+ let output_array = match $eval_mode {
+ EvalMode::Ansi => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let divisor = 10_i128.pow($scale as u32);
+ let (truncated, decimal) = (value / divisor, (value % divisor).abs());
+ let is_overflow = truncated.abs() > i32::MAX.into();
+ if is_overflow {
+ return Err(cast_overflow(
+ &format!("{}.{}BD", truncated, decimal),
+ &format!("DECIMAL({},{})", $precision, $scale),
+ $dest_type_str,
+ ));
+ }
+ let i32_value = truncated as i32;
+ <$rust_dest_type>::try_from(i32_value)
+ .map_err(|_| {
+ cast_overflow(
+ &format!("{}.{}BD", truncated, decimal),
+ &format!("DECIMAL({},{})", $precision, $scale),
+ $dest_type_str,
+ )
+ })
+ .map(Some)
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ _ => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let divisor = 10_i128.pow($scale as u32);
+ let i32_value = (value / divisor) as i32;
+ Ok::, SparkError>(Some(
+ i32_value as $rust_dest_type,
+ ))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ };
+ Ok(Arc::new(output_array) as ArrayRef)
+ }};
+}
+
+macro_rules! cast_decimal_to_int32_up {
+ (
+ $array:expr,
+ $eval_mode:expr,
+ $dest_array_type:ty,
+ $rust_dest_type:ty,
+ $dest_type_str:expr,
+ $max_dest_val:expr,
+ $precision:expr,
+ $scale:expr
+ ) => {{
+ let cast_array = $array
+ .as_any()
+ .downcast_ref::()
+ .expect(concat!("Expected a Decimal128ArrayType"));
+
+ let output_array = match $eval_mode {
+ EvalMode::Ansi => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let divisor = 10_i128.pow($scale as u32);
+ let (truncated, decimal) = (value / divisor, (value % divisor).abs());
+ let is_overflow = truncated.abs() > $max_dest_val.into();
+ if is_overflow {
+ return Err(cast_overflow(
+ &format!("{}.{}BD", truncated, decimal),
+ &format!("DECIMAL({},{})", $precision, $scale),
+ $dest_type_str,
+ ));
+ }
+ Ok(Some(truncated as $rust_dest_type))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ _ => cast_array
+ .iter()
+ .map(|value| match value {
+ Some(value) => {
+ let divisor = 10_i128.pow($scale as u32);
+ let truncated = value / divisor;
+ Ok::, SparkError>(Some(
+ truncated as $rust_dest_type,
+ ))
+ }
+ None => Ok(None),
+ })
+ .collect::>()?,
+ };
+ Ok(Arc::new(output_array) as ArrayRef)
+ }};
+}
+
+/// Spark cast options
+#[derive(Debug, Clone, Hash, PartialEq, Eq)]
+pub struct SparkParquetOptions {
+ /// Spark evaluation mode
+ pub eval_mode: EvalMode,
+ /// When cast from/to timezone related types, we need timezone, which will be resolved with
+ /// session local timezone by an analyzer in Spark.
+ // TODO we should change timezone to Tz to avoid repeated parsing
+ pub timezone: String,
+ /// Allow casts that are supported but not guaranteed to be 100% compatible
+ pub allow_incompat: bool,
+ /// Support casting unsigned ints to signed ints (used by Parquet SchemaAdapter)
+ pub allow_cast_unsigned_ints: bool,
+ /// We also use the cast logic for adapting Parquet schemas, so this flag is used
+ /// for that use case
+ pub is_adapting_schema: bool,
+ /// Whether to always represent decimals using 128 bits. If false, the native reader may represent decimals using 32 or 64 bits, depending on the precision.
+ pub use_decimal_128: bool,
+ /// Whether to read dates/timestamps that were written in the legacy hybrid Julian + Gregorian calendar as it is. If false, throw exceptions instead. If the spark type is TimestampNTZ, this should be true.
+ pub use_legacy_date_timestamp_or_ntz: bool,
+}
+
+impl SparkParquetOptions {
+ pub fn new(eval_mode: EvalMode, timezone: &str, allow_incompat: bool) -> Self {
+ Self {
+ eval_mode,
+ timezone: timezone.to_string(),
+ allow_incompat,
+ allow_cast_unsigned_ints: false,
+ is_adapting_schema: false,
+ use_decimal_128: false,
+ use_legacy_date_timestamp_or_ntz: false,
+ }
+ }
+
+ pub fn new_without_timezone(eval_mode: EvalMode, allow_incompat: bool) -> Self {
+ Self {
+ eval_mode,
+ timezone: "".to_string(),
+ allow_incompat,
+ allow_cast_unsigned_ints: false,
+ is_adapting_schema: false,
+ use_decimal_128: false,
+ use_legacy_date_timestamp_or_ntz: false,
+ }
+ }
+}
+
+/// Spark-compatible cast implementation. Defers to DataFusion's cast where that is known
+/// to be compatible, and returns an error when a not supported and not DF-compatible cast
+/// is requested.
+pub fn spark_parquet_convert(
+ arg: ColumnarValue,
+ data_type: &DataType,
+ parquet_options: &SparkParquetOptions,
+) -> DataFusionResult {
+ match arg {
+ ColumnarValue::Array(array) => Ok(ColumnarValue::Array(cast_array(
+ array,
+ data_type,
+ parquet_options,
+ )?)),
+ ColumnarValue::Scalar(scalar) => {
+ // Note that normally CAST(scalar) should be fold in Spark JVM side. However, for
+ // some cases e.g., scalar subquery, Spark will not fold it, so we need to handle it
+ // here.
+ let array = scalar.to_array()?;
+ let scalar =
+ ScalarValue::try_from_array(&cast_array(array, data_type, parquet_options)?, 0)?;
+ Ok(ColumnarValue::Scalar(scalar))
+ }
+ }
+}
+
+fn cast_array(
+ array: ArrayRef,
+ to_type: &DataType,
+ parquet_options: &SparkParquetOptions,
+) -> DataFusionResult {
+ use DataType::*;
+ let array = array_with_timezone(array, parquet_options.timezone.clone(), Some(to_type))?;
+ let from_type = array.data_type().clone();
+
+ let array = match &from_type {
+ Dictionary(key_type, value_type)
+ if key_type.as_ref() == &Int32
+ && (value_type.as_ref() == &Utf8 || value_type.as_ref() == &LargeUtf8) =>
+ {
+ let dict_array = array
+ .as_any()
+ .downcast_ref::>()
+ .expect("Expected a dictionary array");
+
+ let casted_dictionary = DictionaryArray::::new(
+ dict_array.keys().clone(),
+ cast_array(Arc::clone(dict_array.values()), to_type, parquet_options)?,
+ );
+
+ let casted_result = match to_type {
+ Dictionary(_, _) => Arc::new(casted_dictionary.clone()),
+ _ => take(casted_dictionary.values().as_ref(), dict_array.keys(), None)?,
+ };
+ return Ok(spark_cast_postprocess(casted_result, &from_type, to_type));
+ }
+ _ => array,
+ };
+ let from_type = array.data_type();
+ let eval_mode = parquet_options.eval_mode;
+
+ let cast_result = match (from_type, to_type) {
+ (Utf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode),
+ (LargeUtf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode),
+ (Utf8, Timestamp(_, _)) => {
+ cast_string_to_timestamp(&array, to_type, eval_mode, &parquet_options.timezone)
+ }
+ (Utf8, Date32) => cast_string_to_date(&array, to_type, eval_mode),
+ (Int64, Int32)
+ | (Int64, Int16)
+ | (Int64, Int8)
+ | (Int32, Int16)
+ | (Int32, Int8)
+ | (Int16, Int8)
+ if eval_mode != EvalMode::Try =>
+ {
+ spark_cast_int_to_int(&array, eval_mode, from_type, to_type)
+ }
+ (Utf8, Int8 | Int16 | Int32 | Int64) => {
+ cast_string_to_int::(to_type, &array, eval_mode)
+ }
+ (LargeUtf8, Int8 | Int16 | Int32 | Int64) => {
+ cast_string_to_int::(to_type, &array, eval_mode)
+ }
+ (Float64, Utf8) => spark_cast_float64_to_utf8::(&array, eval_mode),
+ (Float64, LargeUtf8) => spark_cast_float64_to_utf8::(&array, eval_mode),
+ (Float32, Utf8) => spark_cast_float32_to_utf8::(&array, eval_mode),
+ (Float32, LargeUtf8) => spark_cast_float32_to_utf8::(&array, eval_mode),
+ (Float32, Decimal128(precision, scale)) => {
+ cast_float32_to_decimal128(&array, *precision, *scale, eval_mode)
+ }
+ (Float64, Decimal128(precision, scale)) => {
+ cast_float64_to_decimal128(&array, *precision, *scale, eval_mode)
+ }
+ (Float32, Int8)
+ | (Float32, Int16)
+ | (Float32, Int32)
+ | (Float32, Int64)
+ | (Float64, Int8)
+ | (Float64, Int16)
+ | (Float64, Int32)
+ | (Float64, Int64)
+ | (Decimal128(_, _), Int8)
+ | (Decimal128(_, _), Int16)
+ | (Decimal128(_, _), Int32)
+ | (Decimal128(_, _), Int64)
+ if eval_mode != EvalMode::Try =>
+ {
+ spark_cast_nonintegral_numeric_to_integral(&array, eval_mode, from_type, to_type)
+ }
+ (Struct(_), Utf8) => Ok(casts_struct_to_string(array.as_struct(), parquet_options)?),
+ (Struct(_), Struct(_)) => Ok(cast_struct_to_struct(
+ array.as_struct(),
+ from_type,
+ to_type,
+ parquet_options,
+ )?),
+ (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64)
+ if parquet_options.allow_cast_unsigned_ints =>
+ {
+ Ok(cast_with_options(&array, to_type, &PARQUET_OPTIONS)?)
+ }
+ _ if parquet_options.is_adapting_schema
+ || is_datafusion_spark_compatible(
+ from_type,
+ to_type,
+ parquet_options.allow_incompat,
+ ) =>
+ {
+ // use DataFusion cast only when we know that it is compatible with Spark
+ Ok(cast_with_options(&array, to_type, &PARQUET_OPTIONS)?)
+ }
+ _ => {
+ // we should never reach this code because the Scala code should be checking
+ // for supported cast operations and falling back to Spark for anything that
+ // is not yet supported
+ Err(SparkError::Internal(format!(
+ "Native cast invoked for unsupported cast from {from_type:?} to {to_type:?}"
+ )))
+ }
+ };
+ Ok(spark_cast_postprocess(cast_result?, from_type, to_type))
+}
+
+/// Determines if DataFusion supports the given cast in a way that is
+/// compatible with Spark
+fn is_datafusion_spark_compatible(
+ from_type: &DataType,
+ to_type: &DataType,
+ allow_incompat: bool,
+) -> bool {
+ if from_type == to_type {
+ return true;
+ }
+ match from_type {
+ DataType::Boolean => matches!(
+ to_type,
+ DataType::Int8
+ | DataType::Int16
+ | DataType::Int32
+ | DataType::Int64
+ | DataType::Float32
+ | DataType::Float64
+ | DataType::Utf8
+ ),
+ DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => {
+ // note that the cast from Int32/Int64 -> Decimal128 here is actually
+ // not compatible with Spark (no overflow checks) but we have tests that
+ // rely on this cast working so we have to leave it here for now
+ matches!(
+ to_type,
+ DataType::Boolean
+ | DataType::Int8
+ | DataType::Int16
+ | DataType::Int32
+ | DataType::Int64
+ | DataType::Float32
+ | DataType::Float64
+ | DataType::Decimal128(_, _)
+ | DataType::Utf8
+ )
+ }
+ DataType::Float32 | DataType::Float64 => matches!(
+ to_type,
+ DataType::Boolean
+ | DataType::Int8
+ | DataType::Int16
+ | DataType::Int32
+ | DataType::Int64
+ | DataType::Float32
+ | DataType::Float64
+ ),
+ DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => matches!(
+ to_type,
+ DataType::Int8
+ | DataType::Int16
+ | DataType::Int32
+ | DataType::Int64
+ | DataType::Float32
+ | DataType::Float64
+ | DataType::Decimal128(_, _)
+ | DataType::Decimal256(_, _)
+ | DataType::Utf8 // note that there can be formatting differences
+ ),
+ DataType::Utf8 if allow_incompat => matches!(
+ to_type,
+ DataType::Binary | DataType::Float32 | DataType::Float64 | DataType::Decimal128(_, _)
+ ),
+ DataType::Utf8 => matches!(to_type, DataType::Binary),
+ DataType::Date32 => matches!(to_type, DataType::Utf8),
+ DataType::Timestamp(_, _) => {
+ matches!(
+ to_type,
+ DataType::Int64 | DataType::Date32 | DataType::Utf8 | DataType::Timestamp(_, _)
+ )
+ }
+ DataType::Binary => {
+ // note that this is not completely Spark compatible because
+ // DataFusion only supports binary data containing valid UTF-8 strings
+ matches!(to_type, DataType::Utf8)
+ }
+ _ => false,
+ }
+}
+
+/// Cast between struct types based on logic in
+/// `org.apache.spark.sql.catalyst.expressions.Cast#castStruct`.
+fn cast_struct_to_struct(
+ array: &StructArray,
+ from_type: &DataType,
+ to_type: &DataType,
+ parquet_options: &SparkParquetOptions,
+) -> DataFusionResult {
+ match (from_type, to_type) {
+ (DataType::Struct(from_fields), DataType::Struct(to_fields)) => {
+ // TODO some of this logic may be specific to converting Parquet to Spark
+ let mut field_name_to_index_map = HashMap::new();
+ for (i, field) in from_fields.iter().enumerate() {
+ field_name_to_index_map.insert(field.name(), i);
+ }
+ assert_eq!(field_name_to_index_map.len(), from_fields.len());
+ let mut cast_fields: Vec = Vec::with_capacity(to_fields.len());
+ for i in 0..to_fields.len() {
+ let from_index = field_name_to_index_map[to_fields[i].name()];
+ let cast_field = cast_array(
+ Arc::clone(array.column(from_index)),
+ to_fields[i].data_type(),
+ parquet_options,
+ )?;
+ cast_fields.push(cast_field);
+ }
+ Ok(Arc::new(StructArray::new(
+ to_fields.clone(),
+ cast_fields,
+ array.nulls().cloned(),
+ )))
+ }
+ _ => unreachable!(),
+ }
+}
+
+fn casts_struct_to_string(
+ array: &StructArray,
+ parquet_options: &SparkParquetOptions,
+) -> DataFusionResult {
+ // cast each field to a string
+ let string_arrays: Vec = array
+ .columns()
+ .iter()
+ .map(|arr| {
+ spark_parquet_convert(
+ ColumnarValue::Array(Arc::clone(arr)),
+ &DataType::Utf8,
+ parquet_options,
+ )
+ .and_then(|cv| cv.into_array(arr.len()))
+ })
+ .collect::>>()?;
+ let string_arrays: Vec<&StringArray> =
+ string_arrays.iter().map(|arr| arr.as_string()).collect();
+ // build the struct string containing entries in the format `"field_name":field_value`
+ let mut builder = StringBuilder::with_capacity(array.len(), array.len() * 16);
+ let mut str = String::with_capacity(array.len() * 16);
+ for row_index in 0..array.len() {
+ if array.is_null(row_index) {
+ builder.append_null();
+ } else {
+ str.clear();
+ let mut any_fields_written = false;
+ str.push('{');
+ for field in &string_arrays {
+ if any_fields_written {
+ str.push_str(", ");
+ }
+ if field.is_null(row_index) {
+ str.push_str("null");
+ } else {
+ str.push_str(field.value(row_index));
+ }
+ any_fields_written = true;
+ }
+ str.push('}');
+ builder.append_value(&str);
+ }
+ }
+ Ok(Arc::new(builder.finish()))
+}
+
+fn cast_string_to_int(
+ to_type: &DataType,
+ array: &ArrayRef,
+ eval_mode: EvalMode,
+) -> SparkResult {
+ let string_array = array
+ .as_any()
+ .downcast_ref::>()
+ .expect("cast_string_to_int expected a string array");
+
+ let cast_array: ArrayRef = match to_type {
+ DataType::Int8 => cast_utf8_to_int!(string_array, eval_mode, Int8Type, cast_string_to_i8)?,
+ DataType::Int16 => {
+ cast_utf8_to_int!(string_array, eval_mode, Int16Type, cast_string_to_i16)?
+ }
+ DataType::Int32 => {
+ cast_utf8_to_int!(string_array, eval_mode, Int32Type, cast_string_to_i32)?
+ }
+ DataType::Int64 => {
+ cast_utf8_to_int!(string_array, eval_mode, Int64Type, cast_string_to_i64)?
+ }
+ dt => unreachable!(
+ "{}",
+ format!("invalid integer type {dt} in cast from string")
+ ),
+ };
+ Ok(cast_array)
+}
+
+fn cast_string_to_date(
+ array: &ArrayRef,
+ to_type: &DataType,
+ eval_mode: EvalMode,
+) -> SparkResult {
+ let string_array = array
+ .as_any()
+ .downcast_ref::>()
+ .expect("Expected a string array");
+
+ if to_type != &DataType::Date32 {
+ unreachable!("Invalid data type {:?} in cast from string", to_type);
+ }
+
+ let len = string_array.len();
+ let mut cast_array = PrimitiveArray::::builder(len);
+
+ for i in 0..len {
+ let value = if string_array.is_null(i) {
+ None
+ } else {
+ match date_parser(string_array.value(i), eval_mode) {
+ Ok(Some(cast_value)) => Some(cast_value),
+ Ok(None) => None,
+ Err(e) => return Err(e),
+ }
+ };
+
+ match value {
+ Some(cast_value) => cast_array.append_value(cast_value),
+ None => cast_array.append_null(),
+ }
+ }
+
+ Ok(Arc::new(cast_array.finish()) as ArrayRef)
+}
+
+fn cast_string_to_timestamp(
+ array: &ArrayRef,
+ to_type: &DataType,
+ eval_mode: EvalMode,
+ timezone_str: &str,
+) -> SparkResult {
+ let string_array = array
+ .as_any()
+ .downcast_ref::>()
+ .expect("Expected a string array");
+
+ let tz = &timezone::Tz::from_str(timezone_str).unwrap();
+
+ let cast_array: ArrayRef = match to_type {
+ DataType::Timestamp(_, _) => {
+ cast_utf8_to_timestamp!(
+ string_array,
+ eval_mode,
+ TimestampMicrosecondType,
+ timestamp_parser,
+ tz
+ )
+ }
+ _ => unreachable!("Invalid data type {:?} in cast from string", to_type),
+ };
+ Ok(cast_array)
+}
+
+fn cast_float64_to_decimal128(
+ array: &dyn Array,
+ precision: u8,
+ scale: i8,
+ eval_mode: EvalMode,
+) -> SparkResult {
+ cast_floating_point_to_decimal128::(array, precision, scale, eval_mode)
+}
+
+fn cast_float32_to_decimal128(
+ array: &dyn Array,
+ precision: u8,
+ scale: i8,
+ eval_mode: EvalMode,
+) -> SparkResult {
+ cast_floating_point_to_decimal128::(array, precision, scale, eval_mode)
+}
+
+fn cast_floating_point_to_decimal128(
+ array: &dyn Array,
+ precision: u8,
+ scale: i8,
+ eval_mode: EvalMode,
+) -> SparkResult
+where
+ ::Native: AsPrimitive,
+{
+ let input = array.as_any().downcast_ref::>().unwrap();
+ let mut cast_array = PrimitiveArray::::builder(input.len());
+
+ let mul = 10_f64.powi(scale as i32);
+
+ for i in 0..input.len() {
+ if input.is_null(i) {
+ cast_array.append_null();
+ } else {
+ let input_value = input.value(i).as_();
+ let value = (input_value * mul).round().to_i128();
+
+ match value {
+ Some(v) => {
+ if Decimal128Type::validate_decimal_precision(v, precision).is_err() {
+ if eval_mode == EvalMode::Ansi {
+ return Err(SparkError::NumericValueOutOfRange {
+ value: input_value.to_string(),
+ precision,
+ scale,
+ });
+ } else {
+ cast_array.append_null();
+ }
+ }
+ cast_array.append_value(v);
+ }
+ None => {
+ if eval_mode == EvalMode::Ansi {
+ return Err(SparkError::NumericValueOutOfRange {
+ value: input_value.to_string(),
+ precision,
+ scale,
+ });
+ } else {
+ cast_array.append_null();
+ }
+ }
+ }
+ }
+ }
+
+ let res = Arc::new(
+ cast_array
+ .with_precision_and_scale(precision, scale)?
+ .finish(),
+ ) as ArrayRef;
+ Ok(res)
+}
+
+fn spark_cast_float64_to_utf8(
+ from: &dyn Array,
+ _eval_mode: EvalMode,
+) -> SparkResult
+where
+ OffsetSize: OffsetSizeTrait,
+{
+ cast_float_to_string!(from, _eval_mode, f64, Float64Array, OffsetSize)
+}
+
+fn spark_cast_float32_to_utf8(
+ from: &dyn Array,
+ _eval_mode: EvalMode,
+) -> SparkResult
+where
+ OffsetSize: OffsetSizeTrait,
+{
+ cast_float_to_string!(from, _eval_mode, f32, Float32Array, OffsetSize)
+}
+
+fn spark_cast_int_to_int(
+ array: &dyn Array,
+ eval_mode: EvalMode,
+ from_type: &DataType,
+ to_type: &DataType,
+) -> SparkResult {
+ match (from_type, to_type) {
+ (DataType::Int64, DataType::Int32) => cast_int_to_int_macro!(
+ array, eval_mode, Int64Type, Int32Type, from_type, i32, "BIGINT", "INT"
+ ),
+ (DataType::Int64, DataType::Int16) => cast_int_to_int_macro!(
+ array, eval_mode, Int64Type, Int16Type, from_type, i16, "BIGINT", "SMALLINT"
+ ),
+ (DataType::Int64, DataType::Int8) => cast_int_to_int_macro!(
+ array, eval_mode, Int64Type, Int8Type, from_type, i8, "BIGINT", "TINYINT"
+ ),
+ (DataType::Int32, DataType::Int16) => cast_int_to_int_macro!(
+ array, eval_mode, Int32Type, Int16Type, from_type, i16, "INT", "SMALLINT"
+ ),
+ (DataType::Int32, DataType::Int8) => cast_int_to_int_macro!(
+ array, eval_mode, Int32Type, Int8Type, from_type, i8, "INT", "TINYINT"
+ ),
+ (DataType::Int16, DataType::Int8) => cast_int_to_int_macro!(
+ array, eval_mode, Int16Type, Int8Type, from_type, i8, "SMALLINT", "TINYINT"
+ ),
+ _ => unreachable!(
+ "{}",
+ format!("invalid integer type {to_type} in cast from {from_type}")
+ ),
+ }
+}
+
+fn spark_cast_utf8_to_boolean(
+ from: &dyn Array,
+ eval_mode: EvalMode,
+) -> SparkResult
+where
+ OffsetSize: OffsetSizeTrait,
+{
+ let array = from
+ .as_any()
+ .downcast_ref::>()
+ .unwrap();
+
+ let output_array = array
+ .iter()
+ .map(|value| match value {
+ Some(value) => match value.to_ascii_lowercase().trim() {
+ "t" | "true" | "y" | "yes" | "1" => Ok(Some(true)),
+ "f" | "false" | "n" | "no" | "0" => Ok(Some(false)),
+ _ if eval_mode == EvalMode::Ansi => Err(SparkError::CastInvalidValue {
+ value: value.to_string(),
+ from_type: "STRING".to_string(),
+ to_type: "BOOLEAN".to_string(),
+ }),
+ _ => Ok(None),
+ },
+ _ => Ok(None),
+ })
+ .collect::>()?;
+
+ Ok(Arc::new(output_array))
+}
+
+fn spark_cast_nonintegral_numeric_to_integral(
+ array: &dyn Array,
+ eval_mode: EvalMode,
+ from_type: &DataType,
+ to_type: &DataType,
+) -> SparkResult {
+ match (from_type, to_type) {
+ (DataType::Float32, DataType::Int8) => cast_float_to_int16_down!(
+ array,
+ eval_mode,
+ Float32Array,
+ Int8Array,
+ f32,
+ i8,
+ "FLOAT",
+ "TINYINT",
+ "{:e}"
+ ),
+ (DataType::Float32, DataType::Int16) => cast_float_to_int16_down!(
+ array,
+ eval_mode,
+ Float32Array,
+ Int16Array,
+ f32,
+ i16,
+ "FLOAT",
+ "SMALLINT",
+ "{:e}"
+ ),
+ (DataType::Float32, DataType::Int32) => cast_float_to_int32_up!(
+ array,
+ eval_mode,
+ Float32Array,
+ Int32Array,
+ f32,
+ i32,
+ "FLOAT",
+ "INT",
+ i32::MAX,
+ "{:e}"
+ ),
+ (DataType::Float32, DataType::Int64) => cast_float_to_int32_up!(
+ array,
+ eval_mode,
+ Float32Array,
+ Int64Array,
+ f32,
+ i64,
+ "FLOAT",
+ "BIGINT",
+ i64::MAX,
+ "{:e}"
+ ),
+ (DataType::Float64, DataType::Int8) => cast_float_to_int16_down!(
+ array,
+ eval_mode,
+ Float64Array,
+ Int8Array,
+ f64,
+ i8,
+ "DOUBLE",
+ "TINYINT",
+ "{:e}D"
+ ),
+ (DataType::Float64, DataType::Int16) => cast_float_to_int16_down!(
+ array,
+ eval_mode,
+ Float64Array,
+ Int16Array,
+ f64,
+ i16,
+ "DOUBLE",
+ "SMALLINT",
+ "{:e}D"
+ ),
+ (DataType::Float64, DataType::Int32) => cast_float_to_int32_up!(
+ array,
+ eval_mode,
+ Float64Array,
+ Int32Array,
+ f64,
+ i32,
+ "DOUBLE",
+ "INT",
+ i32::MAX,
+ "{:e}D"
+ ),
+ (DataType::Float64, DataType::Int64) => cast_float_to_int32_up!(
+ array,
+ eval_mode,
+ Float64Array,
+ Int64Array,
+ f64,
+ i64,
+ "DOUBLE",
+ "BIGINT",
+ i64::MAX,
+ "{:e}D"
+ ),
+ (DataType::Decimal128(precision, scale), DataType::Int8) => {
+ cast_decimal_to_int16_down!(
+ array, eval_mode, Int8Array, i8, "TINYINT", precision, *scale
+ )
+ }
+ (DataType::Decimal128(precision, scale), DataType::Int16) => {
+ cast_decimal_to_int16_down!(
+ array, eval_mode, Int16Array, i16, "SMALLINT", precision, *scale
+ )
+ }
+ (DataType::Decimal128(precision, scale), DataType::Int32) => {
+ cast_decimal_to_int32_up!(
+ array,
+ eval_mode,
+ Int32Array,
+ i32,
+ "INT",
+ i32::MAX,
+ *precision,
+ *scale
+ )
+ }
+ (DataType::Decimal128(precision, scale), DataType::Int64) => {
+ cast_decimal_to_int32_up!(
+ array,
+ eval_mode,
+ Int64Array,
+ i64,
+ "BIGINT",
+ i64::MAX,
+ *precision,
+ *scale
+ )
+ }
+ _ => unreachable!(
+ "{}",
+ format!("invalid cast from non-integral numeric type: {from_type} to integral numeric type: {to_type}")
+ ),
+ }
+}
+
+/// Equivalent to org.apache.spark.unsafe.types.UTF8String.toByte
+fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> SparkResult> {
+ Ok(cast_string_to_int_with_range_check(
+ str,
+ eval_mode,
+ "TINYINT",
+ i8::MIN as i32,
+ i8::MAX as i32,
+ )?
+ .map(|v| v as i8))
+}
+
+/// Equivalent to org.apache.spark.unsafe.types.UTF8String.toShort
+fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> SparkResult > {
+ Ok(cast_string_to_int_with_range_check(
+ str,
+ eval_mode,
+ "SMALLINT",
+ i16::MIN as i32,
+ i16::MAX as i32,
+ )?
+ .map(|v| v as i16))
+}
+
+/// Equivalent to org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper)
+fn cast_string_to_i32(str: &str, eval_mode: EvalMode) -> SparkResult > {
+ do_cast_string_to_int::(str, eval_mode, "INT", i32::MIN)
+}
+
+/// Equivalent to org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper intWrapper)
+fn cast_string_to_i64(str: &str, eval_mode: EvalMode) -> SparkResult> {
+ do_cast_string_to_int::(str, eval_mode, "BIGINT", i64::MIN)
+}
+
+fn cast_string_to_int_with_range_check(
+ str: &str,
+ eval_mode: EvalMode,
+ type_name: &str,
+ min: i32,
+ max: i32,
+) -> SparkResult> {
+ match do_cast_string_to_int(str, eval_mode, type_name, i32::MIN)? {
+ None => Ok(None),
+ Some(v) if v >= min && v <= max => Ok(Some(v)),
+ _ if eval_mode == EvalMode::Ansi => Err(invalid_value(str, "STRING", type_name)),
+ _ => Ok(None),
+ }
+}
+
+/// Equivalent to
+/// - org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper, boolean allowDecimal)
+/// - org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper longWrapper, boolean allowDecimal)
+fn do_cast_string_to_int<
+ T: Num + PartialOrd + Integer + CheckedSub + CheckedNeg + From + Copy,
+>(
+ str: &str,
+ eval_mode: EvalMode,
+ type_name: &str,
+ min_value: T,
+) -> SparkResult> {
+ let trimmed_str = str.trim();
+ if trimmed_str.is_empty() {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ let len = trimmed_str.len();
+ let mut result: T = T::zero();
+ let mut negative = false;
+ let radix = T::from(10);
+ let stop_value = min_value / radix;
+ let mut parse_sign_and_digits = true;
+
+ for (i, ch) in trimmed_str.char_indices() {
+ if parse_sign_and_digits {
+ if i == 0 {
+ negative = ch == '-';
+ let positive = ch == '+';
+ if negative || positive {
+ if i + 1 == len {
+ // input string is just "+" or "-"
+ return none_or_err(eval_mode, type_name, str);
+ }
+ // consume this char
+ continue;
+ }
+ }
+
+ if ch == '.' {
+ if eval_mode == EvalMode::Legacy {
+ // truncate decimal in legacy mode
+ parse_sign_and_digits = false;
+ continue;
+ } else {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ }
+
+ let digit = if ch.is_ascii_digit() {
+ (ch as u32) - ('0' as u32)
+ } else {
+ return none_or_err(eval_mode, type_name, str);
+ };
+
+ // We are going to process the new digit and accumulate the result. However, before
+ // doing this, if the result is already smaller than the
+ // stopValue(Integer.MIN_VALUE / radix), then result * 10 will definitely be
+ // smaller than minValue, and we can stop
+ if result < stop_value {
+ return none_or_err(eval_mode, type_name, str);
+ }
+
+ // Since the previous result is greater than or equal to stopValue(Integer.MIN_VALUE /
+ // radix), we can just use `result > 0` to check overflow. If result
+ // overflows, we should stop
+ let v = result * radix;
+ let digit = (digit as i32).into();
+ match v.checked_sub(&digit) {
+ Some(x) if x <= T::zero() => result = x,
+ _ => {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ }
+ } else {
+ // make sure fractional digits are valid digits but ignore them
+ if !ch.is_ascii_digit() {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ }
+ }
+
+ if !negative {
+ if let Some(neg) = result.checked_neg() {
+ if neg < T::zero() {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ result = neg;
+ } else {
+ return none_or_err(eval_mode, type_name, str);
+ }
+ }
+
+ Ok(Some(result))
+}
+
+/// Either return Ok(None) or Err(SparkError::CastInvalidValue) depending on the evaluation mode
+#[inline]
+fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> SparkResult> {
+ match eval_mode {
+ EvalMode::Ansi => Err(invalid_value(str, "STRING", type_name)),
+ _ => Ok(None),
+ }
+}
+
+#[inline]
+fn invalid_value(value: &str, from_type: &str, to_type: &str) -> SparkError {
+ SparkError::CastInvalidValue {
+ value: value.to_string(),
+ from_type: from_type.to_string(),
+ to_type: to_type.to_string(),
+ }
+}
+
+#[inline]
+fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> SparkError {
+ SparkError::CastOverFlow {
+ value: value.to_string(),
+ from_type: from_type.to_string(),
+ to_type: to_type.to_string(),
+ }
+}
+
+fn timestamp_parser(
+ value: &str,
+ eval_mode: EvalMode,
+ tz: &T,
+) -> SparkResult> {
+ let value = value.trim();
+ if value.is_empty() {
+ return Ok(None);
+ }
+ // Define regex patterns and corresponding parsing functions
+ let patterns = &[
+ (
+ Regex::new(r"^\d{4,5}$").unwrap(),
+ parse_str_to_year_timestamp as fn(&str, &T) -> SparkResult >,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}$").unwrap(),
+ parse_str_to_month_timestamp,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}-\d{2}$").unwrap(),
+ parse_str_to_day_timestamp,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{1,2}$").unwrap(),
+ parse_str_to_hour_timestamp,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}$").unwrap(),
+ parse_str_to_minute_timestamp,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}$").unwrap(),
+ parse_str_to_second_timestamp,
+ ),
+ (
+ Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{1,6}$").unwrap(),
+ parse_str_to_microsecond_timestamp,
+ ),
+ (
+ Regex::new(r"^T\d{1,2}$").unwrap(),
+ parse_str_to_time_only_timestamp,
+ ),
+ ];
+
+ let mut timestamp = None;
+
+ // Iterate through patterns and try matching
+ for (pattern, parse_func) in patterns {
+ if pattern.is_match(value) {
+ timestamp = parse_func(value, tz)?;
+ break;
+ }
+ }
+
+ if timestamp.is_none() {
+ return if eval_mode == EvalMode::Ansi {
+ Err(SparkError::CastInvalidValue {
+ value: value.to_string(),
+ from_type: "STRING".to_string(),
+ to_type: "TIMESTAMP".to_string(),
+ })
+ } else {
+ Ok(None)
+ };
+ }
+
+ match timestamp {
+ Some(ts) => Ok(Some(ts)),
+ None => Err(SparkError::Internal(
+ "Failed to parse timestamp".to_string(),
+ )),
+ }
+}
+
+fn parse_timestamp_to_micros(
+ timestamp_info: &TimeStampInfo,
+ tz: &T,
+) -> SparkResult> {
+ let datetime = tz.with_ymd_and_hms(
+ timestamp_info.year,
+ timestamp_info.month,
+ timestamp_info.day,
+ timestamp_info.hour,
+ timestamp_info.minute,
+ timestamp_info.second,
+ );
+
+ // Check if datetime is not None
+ let tz_datetime = match datetime.single() {
+ Some(dt) => dt
+ .with_timezone(tz)
+ .with_nanosecond(timestamp_info.microsecond * 1000),
+ None => {
+ return Err(SparkError::Internal(
+ "Failed to parse timestamp".to_string(),
+ ));
+ }
+ };
+
+ let result = match tz_datetime {
+ Some(dt) => dt.timestamp_micros(),
+ None => {
+ return Err(SparkError::Internal(
+ "Failed to parse timestamp".to_string(),
+ ));
+ }
+ };
+
+ Ok(Some(result))
+}
+
+fn get_timestamp_values(
+ value: &str,
+ timestamp_type: &str,
+ tz: &T,
+) -> SparkResult> {
+ let values: Vec<_> = value.split(['T', '-', ':', '.']).collect();
+ let year = values[0].parse::().unwrap_or_default();
+ let month = values.get(1).map_or(1, |m| m.parse::().unwrap_or(1));
+ let day = values.get(2).map_or(1, |d| d.parse::().unwrap_or(1));
+ let hour = values.get(3).map_or(0, |h| h.parse::().unwrap_or(0));
+ let minute = values.get(4).map_or(0, |m| m.parse::().unwrap_or(0));
+ let second = values.get(5).map_or(0, |s| s.parse::().unwrap_or(0));
+ let microsecond = values.get(6).map_or(0, |ms| ms.parse::().unwrap_or(0));
+
+ let mut timestamp_info = TimeStampInfo::default();
+
+ let timestamp_info = match timestamp_type {
+ "year" => timestamp_info.with_year(year),
+ "month" => timestamp_info.with_year(year).with_month(month),
+ "day" => timestamp_info
+ .with_year(year)
+ .with_month(month)
+ .with_day(day),
+ "hour" => timestamp_info
+ .with_year(year)
+ .with_month(month)
+ .with_day(day)
+ .with_hour(hour),
+ "minute" => timestamp_info
+ .with_year(year)
+ .with_month(month)
+ .with_day(day)
+ .with_hour(hour)
+ .with_minute(minute),
+ "second" => timestamp_info
+ .with_year(year)
+ .with_month(month)
+ .with_day(day)
+ .with_hour(hour)
+ .with_minute(minute)
+ .with_second(second),
+ "microsecond" => timestamp_info
+ .with_year(year)
+ .with_month(month)
+ .with_day(day)
+ .with_hour(hour)
+ .with_minute(minute)
+ .with_second(second)
+ .with_microsecond(microsecond),
+ _ => {
+ return Err(SparkError::CastInvalidValue {
+ value: value.to_string(),
+ from_type: "STRING".to_string(),
+ to_type: "TIMESTAMP".to_string(),
+ })
+ }
+ };
+
+ parse_timestamp_to_micros(timestamp_info, tz)
+}
+
+fn parse_str_to_year_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "year", tz)
+}
+
+fn parse_str_to_month_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "month", tz)
+}
+
+fn parse_str_to_day_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "day", tz)
+}
+
+fn parse_str_to_hour_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "hour", tz)
+}
+
+fn parse_str_to_minute_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "minute", tz)
+}
+
+fn parse_str_to_second_timestamp(value: &str, tz: &T) -> SparkResult> {
+ get_timestamp_values(value, "second", tz)
+}
+
+fn parse_str_to_microsecond_timestamp(
+ value: &str,
+ tz: &T,
+) -> SparkResult> {
+ get_timestamp_values(value, "microsecond", tz)
+}
+
+fn parse_str_to_time_only_timestamp(value: &str, tz: &T) -> SparkResult> {
+ let values: Vec<&str> = value.split('T').collect();
+ let time_values: Vec = values[1]
+ .split(':')
+ .map(|v| v.parse::().unwrap_or(0))
+ .collect();
+
+ let datetime = tz.from_utc_datetime(&chrono::Utc::now().naive_utc());
+ let timestamp = datetime
+ .with_timezone(tz)
+ .with_hour(time_values.first().copied().unwrap_or_default())
+ .and_then(|dt| dt.with_minute(*time_values.get(1).unwrap_or(&0)))
+ .and_then(|dt| dt.with_second(*time_values.get(2).unwrap_or(&0)))
+ .and_then(|dt| dt.with_nanosecond(*time_values.get(3).unwrap_or(&0) * 1_000))
+ .map(|dt| dt.timestamp_micros())
+ .unwrap_or_default();
+
+ Ok(Some(timestamp))
+}
+
+//a string to date parser - port of spark's SparkDateTimeUtils#stringToDate.
+fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult> {
+ // local functions
+ fn get_trimmed_start(bytes: &[u8]) -> usize {
+ let mut start = 0;
+ while start < bytes.len() && is_whitespace_or_iso_control(bytes[start]) {
+ start += 1;
+ }
+ start
+ }
+
+ fn get_trimmed_end(start: usize, bytes: &[u8]) -> usize {
+ let mut end = bytes.len() - 1;
+ while end > start && is_whitespace_or_iso_control(bytes[end]) {
+ end -= 1;
+ }
+ end + 1
+ }
+
+ fn is_whitespace_or_iso_control(byte: u8) -> bool {
+ byte.is_ascii_whitespace() || byte.is_ascii_control()
+ }
+
+ fn is_valid_digits(segment: i32, digits: usize) -> bool {
+ // An integer is able to represent a date within [+-]5 million years.
+ let max_digits_year = 7;
+ //year (segment 0) can be between 4 to 7 digits,
+ //month and day (segment 1 and 2) can be between 1 to 2 digits
+ (segment == 0 && digits >= 4 && digits <= max_digits_year)
+ || (segment != 0 && digits > 0 && digits <= 2)
+ }
+
+ fn return_result(date_str: &str, eval_mode: EvalMode) -> SparkResult > {
+ if eval_mode == EvalMode::Ansi {
+ Err(SparkError::CastInvalidValue {
+ value: date_str.to_string(),
+ from_type: "STRING".to_string(),
+ to_type: "DATE".to_string(),
+ })
+ } else {
+ Ok(None)
+ }
+ }
+ // end local functions
+
+ if date_str.is_empty() {
+ return return_result(date_str, eval_mode);
+ }
+
+ //values of date segments year, month and day defaulting to 1
+ let mut date_segments = [1, 1, 1];
+ let mut sign = 1;
+ let mut current_segment = 0;
+ let mut current_segment_value = Wrapping(0);
+ let mut current_segment_digits = 0;
+ let bytes = date_str.as_bytes();
+
+ let mut j = get_trimmed_start(bytes);
+ let str_end_trimmed = get_trimmed_end(j, bytes);
+
+ if j == str_end_trimmed {
+ return return_result(date_str, eval_mode);
+ }
+
+ //assign a sign to the date
+ if bytes[j] == b'-' || bytes[j] == b'+' {
+ sign = if bytes[j] == b'-' { -1 } else { 1 };
+ j += 1;
+ }
+
+ //loop to the end of string until we have processed 3 segments,
+ //exit loop on encountering any space ' ' or 'T' after the 3rd segment
+ while j < str_end_trimmed && (current_segment < 3 && !(bytes[j] == b' ' || bytes[j] == b'T')) {
+ let b = bytes[j];
+ if current_segment < 2 && b == b'-' {
+ //check for validity of year and month segments if current byte is separator
+ if !is_valid_digits(current_segment, current_segment_digits) {
+ return return_result(date_str, eval_mode);
+ }
+ //if valid update corresponding segment with the current segment value.
+ date_segments[current_segment as usize] = current_segment_value.0;
+ current_segment_value = Wrapping(0);
+ current_segment_digits = 0;
+ current_segment += 1;
+ } else if !b.is_ascii_digit() {
+ return return_result(date_str, eval_mode);
+ } else {
+ //increment value of current segment by the next digit
+ let parsed_value = Wrapping((b - b'0') as i32);
+ current_segment_value = current_segment_value * Wrapping(10) + parsed_value;
+ current_segment_digits += 1;
+ }
+ j += 1;
+ }
+
+ //check for validity of last segment
+ if !is_valid_digits(current_segment, current_segment_digits) {
+ return return_result(date_str, eval_mode);
+ }
+
+ if current_segment < 2 && j < str_end_trimmed {
+ // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed.
+ return return_result(date_str, eval_mode);
+ }
+
+ date_segments[current_segment as usize] = current_segment_value.0;
+
+ match NaiveDate::from_ymd_opt(
+ sign * date_segments[0],
+ date_segments[1] as u32,
+ date_segments[2] as u32,
+ ) {
+ Some(date) => {
+ let duration_since_epoch = date
+ .signed_duration_since(NaiveDateTime::UNIX_EPOCH.date())
+ .num_days();
+ Ok(Some(duration_since_epoch.to_i32().unwrap()))
+ }
+ None => Ok(None),
+ }
+}
+
+/// This takes for special casting cases of Spark. E.g., Timestamp to Long.
+/// This function runs as a post process of the DataFusion cast(). By the time it arrives here,
+/// Dictionary arrays are already unpacked by the DataFusion cast() since Spark cannot specify
+/// Dictionary as to_type. The from_type is taken before the DataFusion cast() runs in
+/// expressions/cast.rs, so it can be still Dictionary.
+fn spark_cast_postprocess(array: ArrayRef, from_type: &DataType, to_type: &DataType) -> ArrayRef {
+ match (from_type, to_type) {
+ (DataType::Timestamp(_, _), DataType::Int64) => {
+ // See Spark's `Cast` expression
+ unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap()
+ }
+ (DataType::Dictionary(_, value_type), DataType::Int64)
+ if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) =>
+ {
+ // See Spark's `Cast` expression
+ unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap()
+ }
+ (DataType::Timestamp(_, _), DataType::Utf8) => remove_trailing_zeroes(array),
+ (DataType::Dictionary(_, value_type), DataType::Utf8)
+ if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) =>
+ {
+ remove_trailing_zeroes(array)
+ }
+ _ => array,
+ }
+}
+
+/// A fork & modified version of Arrow's `unary_dyn` which is being deprecated
+fn unary_dyn(array: &ArrayRef, op: F) -> Result
+where
+ T: ArrowPrimitiveType,
+ F: Fn(T::Native) -> T::Native,
+{
+ if let Some(d) = array.as_any_dictionary_opt() {
+ let new_values = unary_dyn::(d.values(), op)?;
+ return Ok(Arc::new(d.with_values(Arc::new(new_values))));
+ }
+
+ match array.as_primitive_opt::() {
+ Some(a) if PrimitiveArray::::is_compatible(a.data_type()) => {
+ Ok(Arc::new(unary::(
+ array.as_any().downcast_ref::>().unwrap(),
+ op,
+ )))
+ }
+ _ => Err(ArrowError::NotYetImplemented(format!(
+ "Cannot perform unary operation of type {} on array of type {}",
+ T::DATA_TYPE,
+ array.data_type()
+ ))),
+ }
+}
+
+/// Remove any trailing zeroes in the string if they occur after in the fractional seconds,
+/// to match Spark behavior
+/// example:
+/// "1970-01-01 05:29:59.900" => "1970-01-01 05:29:59.9"
+/// "1970-01-01 05:29:59.990" => "1970-01-01 05:29:59.99"
+/// "1970-01-01 05:29:59.999" => "1970-01-01 05:29:59.999"
+/// "1970-01-01 05:30:00" => "1970-01-01 05:30:00"
+/// "1970-01-01 05:30:00.001" => "1970-01-01 05:30:00.001"
+fn remove_trailing_zeroes(array: ArrayRef) -> ArrayRef {
+ let string_array = as_generic_string_array::(&array).unwrap();
+ let result = string_array
+ .iter()
+ .map(|s| s.map(trim_end))
+ .collect::>();
+ Arc::new(result) as ArrayRef
+}
+
+fn trim_end(s: &str) -> &str {
+ if s.rfind('.').is_some() {
+ s.trim_end_matches('0')
+ } else {
+ s
+ }
+}
+
+#[cfg(test)]
+mod tests {
+ use arrow::datatypes::TimestampMicrosecondType;
+ use arrow_array::StringArray;
+ use arrow_schema::{Field, Fields, TimeUnit};
+ use std::str::FromStr;
+
+ use super::*;
+
+ #[test]
+ #[cfg_attr(miri, ignore)] // test takes too long with miri
+ fn timestamp_parser_test() {
+ let tz = &timezone::Tz::from_str("UTC").unwrap();
+ // write for all formats
+ assert_eq!(
+ timestamp_parser("2020", EvalMode::Legacy, tz).unwrap(),
+ Some(1577836800000000) // this is in milliseconds
+ );
+ assert_eq!(
+ timestamp_parser("2020-01", EvalMode::Legacy, tz).unwrap(),
+ Some(1577836800000000)
+ );
+ assert_eq!(
+ timestamp_parser("2020-01-01", EvalMode::Legacy, tz).unwrap(),
+ Some(1577836800000000)
+ );
+ assert_eq!(
+ timestamp_parser("2020-01-01T12", EvalMode::Legacy, tz).unwrap(),
+ Some(1577880000000000)
+ );
+ assert_eq!(
+ timestamp_parser("2020-01-01T12:34", EvalMode::Legacy, tz).unwrap(),
+ Some(1577882040000000)
+ );
+ assert_eq!(
+ timestamp_parser("2020-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(),
+ Some(1577882096000000)
+ );
+ assert_eq!(
+ timestamp_parser("2020-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(),
+ Some(1577882096123456)
+ );
+ assert_eq!(
+ timestamp_parser("0100", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011459200000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011459200000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01-01", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011459200000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01-01T12", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011416000000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01-01T12:34", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011413960000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011413904000000)
+ );
+ assert_eq!(
+ timestamp_parser("0100-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(),
+ Some(-59011413903876544)
+ );
+ assert_eq!(
+ timestamp_parser("10000", EvalMode::Legacy, tz).unwrap(),
+ Some(253402300800000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01", EvalMode::Legacy, tz).unwrap(),
+ Some(253402300800000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01-01", EvalMode::Legacy, tz).unwrap(),
+ Some(253402300800000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01-01T12", EvalMode::Legacy, tz).unwrap(),
+ Some(253402344000000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01-01T12:34", EvalMode::Legacy, tz).unwrap(),
+ Some(253402346040000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(),
+ Some(253402346096000000)
+ );
+ assert_eq!(
+ timestamp_parser("10000-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(),
+ Some(253402346096123456)
+ );
+ // assert_eq!(
+ // timestamp_parser("T2", EvalMode::Legacy).unwrap(),
+ // Some(1714356000000000) // this value needs to change everyday.
+ // );
+ }
+
+ #[test]
+ #[cfg_attr(miri, ignore)] // test takes too long with miri
+ fn test_cast_string_to_timestamp() {
+ let array: ArrayRef = Arc::new(StringArray::from(vec![
+ Some("2020-01-01T12:34:56.123456"),
+ Some("T2"),
+ Some("0100-01-01T12:34:56.123456"),
+ Some("10000-01-01T12:34:56.123456"),
+ ]));
+ let tz = &timezone::Tz::from_str("UTC").unwrap();
+
+ let string_array = array
+ .as_any()
+ .downcast_ref::>()
+ .expect("Expected a string array");
+
+ let eval_mode = EvalMode::Legacy;
+ let result = cast_utf8_to_timestamp!(
+ &string_array,
+ eval_mode,
+ TimestampMicrosecondType,
+ timestamp_parser,
+ tz
+ );
+
+ assert_eq!(
+ result.data_type(),
+ &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into()))
+ );
+ assert_eq!(result.len(), 4);
+ }
+
+ #[test]
+ fn test_cast_dict_string_to_timestamp() -> DataFusionResult<()> {
+ // prepare input data
+ let keys = Int32Array::from(vec![0, 1]);
+ let values: ArrayRef = Arc::new(StringArray::from(vec![
+ Some("2020-01-01T12:34:56.123456"),
+ Some("T2"),
+ ]));
+ let dict_array = Arc::new(DictionaryArray::new(keys, values));
+
+ let timezone = "UTC".to_string();
+ // test casting string dictionary array to timestamp array
+ let parquet_options = SparkParquetOptions::new(EvalMode::Legacy, &timezone, false);
+ let result = cast_array(
+ dict_array,
+ &DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.clone().into())),
+ &parquet_options,
+ )?;
+ assert_eq!(
+ *result.data_type(),
+ DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.into()))
+ );
+ assert_eq!(result.len(), 2);
+
+ Ok(())
+ }
+
+ #[test]
+ fn date_parser_test() {
+ for date in &[
+ "2020",
+ "2020-01",
+ "2020-01-01",
+ "02020-01-01",
+ "002020-01-01",
+ "0002020-01-01",
+ "2020-1-1",
+ "2020-01-01 ",
+ "2020-01-01T",
+ ] {
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Ansi, EvalMode::Try] {
+ assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(18262));
+ }
+ }
+
+ //dates in invalid formats
+ for date in &[
+ "abc",
+ "",
+ "not_a_date",
+ "3/",
+ "3/12",
+ "3/12/2020",
+ "3/12/2002 T",
+ "202",
+ "2020-010-01",
+ "2020-10-010",
+ "2020-10-010T",
+ "--262143-12-31",
+ "--262143-12-31 ",
+ ] {
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Try] {
+ assert_eq!(date_parser(date, *eval_mode).unwrap(), None);
+ }
+ assert!(date_parser(date, EvalMode::Ansi).is_err());
+ }
+
+ for date in &["-3638-5"] {
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] {
+ assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(-2048160));
+ }
+ }
+
+ //Naive Date only supports years 262142 AD to 262143 BC
+ //returns None for dates out of range supported by Naive Date.
+ for date in &[
+ "-262144-1-1",
+ "262143-01-1",
+ "262143-1-1",
+ "262143-01-1 ",
+ "262143-01-01T ",
+ "262143-1-01T 1234",
+ "-0973250",
+ ] {
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] {
+ assert_eq!(date_parser(date, *eval_mode).unwrap(), None);
+ }
+ }
+ }
+
+ #[test]
+ fn test_cast_string_to_date() {
+ let array: ArrayRef = Arc::new(StringArray::from(vec![
+ Some("2020"),
+ Some("2020-01"),
+ Some("2020-01-01"),
+ Some("2020-01-01T"),
+ ]));
+
+ let result = cast_string_to_date(&array, &DataType::Date32, EvalMode::Legacy).unwrap();
+
+ let date32_array = result
+ .as_any()
+ .downcast_ref::()
+ .unwrap();
+ assert_eq!(date32_array.len(), 4);
+ date32_array
+ .iter()
+ .for_each(|v| assert_eq!(v.unwrap(), 18262));
+ }
+
+ #[test]
+ fn test_cast_string_array_with_valid_dates() {
+ let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![
+ Some("-262143-12-31"),
+ Some("\n -262143-12-31 "),
+ Some("-262143-12-31T \t\n"),
+ Some("\n\t-262143-12-31T\r"),
+ Some("-262143-12-31T 123123123"),
+ Some("\r\n-262143-12-31T \r123123123"),
+ Some("\n -262143-12-31T \n\t"),
+ ]));
+
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] {
+ let result =
+ cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode)
+ .unwrap();
+
+ let date32_array = result
+ .as_any()
+ .downcast_ref::()
+ .unwrap();
+ assert_eq!(result.len(), 7);
+ date32_array
+ .iter()
+ .for_each(|v| assert_eq!(v.unwrap(), -96464928));
+ }
+ }
+
+ #[test]
+ fn test_cast_string_array_with_invalid_dates() {
+ let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![
+ Some("2020"),
+ Some("2020-01"),
+ Some("2020-01-01"),
+ //4 invalid dates
+ Some("2020-010-01T"),
+ Some("202"),
+ Some(" 202 "),
+ Some("\n 2020-\r8 "),
+ Some("2020-01-01T"),
+ // Overflows i32
+ Some("-4607172990231812908"),
+ ]));
+
+ for eval_mode in &[EvalMode::Legacy, EvalMode::Try] {
+ let result =
+ cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode)
+ .unwrap();
+
+ let date32_array = result
+ .as_any()
+ .downcast_ref::()
+ .unwrap();
+ assert_eq!(
+ date32_array.iter().collect::>(),
+ vec![
+ Some(18262),
+ Some(18262),
+ Some(18262),
+ None,
+ None,
+ None,
+ None,
+ Some(18262),
+ None
+ ]
+ );
+ }
+
+ let result =
+ cast_string_to_date(&array_with_invalid_date, &DataType::Date32, EvalMode::Ansi);
+ match result {
+ Err(e) => assert!(
+ e.to_string().contains(
+ "[CAST_INVALID_INPUT] The value '2020-010-01T' of the type \"STRING\" cannot be cast to \"DATE\" because it is malformed")
+ ),
+ _ => panic!("Expected error"),
+ }
+ }
+
+ #[test]
+ fn test_cast_string_as_i8() {
+ // basic
+ assert_eq!(
+ cast_string_to_i8("127", EvalMode::Legacy).unwrap(),
+ Some(127_i8)
+ );
+ assert_eq!(cast_string_to_i8("128", EvalMode::Legacy).unwrap(), None);
+ assert!(cast_string_to_i8("128", EvalMode::Ansi).is_err());
+ // decimals
+ assert_eq!(
+ cast_string_to_i8("0.2", EvalMode::Legacy).unwrap(),
+ Some(0_i8)
+ );
+ assert_eq!(
+ cast_string_to_i8(".", EvalMode::Legacy).unwrap(),
+ Some(0_i8)
+ );
+ // TRY should always return null for decimals
+ assert_eq!(cast_string_to_i8("0.2", EvalMode::Try).unwrap(), None);
+ assert_eq!(cast_string_to_i8(".", EvalMode::Try).unwrap(), None);
+ // ANSI mode should throw error on decimal
+ assert!(cast_string_to_i8("0.2", EvalMode::Ansi).is_err());
+ assert!(cast_string_to_i8(".", EvalMode::Ansi).is_err());
+ }
+
+ #[test]
+ fn test_cast_unsupported_timestamp_to_date() {
+ // Since datafusion uses chrono::Datetime internally not all dates representable by TimestampMicrosecondType are supported
+ let timestamps: PrimitiveArray = vec![i64::MAX].into();
+ let parquet_options = SparkParquetOptions::new(EvalMode::Legacy, "UTC", false);
+ let result = cast_array(
+ Arc::new(timestamps.with_timezone("Europe/Copenhagen")),
+ &DataType::Date32,
+ &parquet_options,
+ );
+ assert!(result.is_err())
+ }
+
+ #[test]
+ fn test_cast_invalid_timezone() {
+ let timestamps: PrimitiveArray = vec![i64::MAX].into();
+ let parquet_options =
+ SparkParquetOptions::new(EvalMode::Legacy, "Not a valid timezone", false);
+ let result = cast_array(
+ Arc::new(timestamps.with_timezone("Europe/Copenhagen")),
+ &DataType::Date32,
+ &parquet_options,
+ );
+ assert!(result.is_err())
+ }
+
+ #[test]
+ fn test_cast_struct_to_utf8() {
+ let a: ArrayRef = Arc::new(Int32Array::from(vec![
+ Some(1),
+ Some(2),
+ None,
+ Some(4),
+ Some(5),
+ ]));
+ let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"]));
+ let c: ArrayRef = Arc::new(StructArray::from(vec![
+ (Arc::new(Field::new("a", DataType::Int32, true)), a),
+ (Arc::new(Field::new("b", DataType::Utf8, true)), b),
+ ]));
+ let string_array = cast_array(
+ c,
+ &DataType::Utf8,
+ &SparkParquetOptions::new(EvalMode::Legacy, "UTC", false),
+ )
+ .unwrap();
+ let string_array = string_array.as_string::();
+ assert_eq!(5, string_array.len());
+ assert_eq!(r#"{1, a}"#, string_array.value(0));
+ assert_eq!(r#"{2, b}"#, string_array.value(1));
+ assert_eq!(r#"{null, c}"#, string_array.value(2));
+ assert_eq!(r#"{4, d}"#, string_array.value(3));
+ assert_eq!(r#"{5, e}"#, string_array.value(4));
+ }
+
+ #[test]
+ fn test_cast_struct_to_struct() {
+ let a: ArrayRef = Arc::new(Int32Array::from(vec![
+ Some(1),
+ Some(2),
+ None,
+ Some(4),
+ Some(5),
+ ]));
+ let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"]));
+ let c: ArrayRef = Arc::new(StructArray::from(vec![
+ (Arc::new(Field::new("a", DataType::Int32, true)), a),
+ (Arc::new(Field::new("b", DataType::Utf8, true)), b),
+ ]));
+ // change type of "a" from Int32 to Utf8
+ let fields = Fields::from(vec![
+ Field::new("a", DataType::Utf8, true),
+ Field::new("b", DataType::Utf8, true),
+ ]);
+ let cast_array = spark_parquet_convert(
+ ColumnarValue::Array(c),
+ &DataType::Struct(fields),
+ &SparkParquetOptions::new(EvalMode::Legacy, "UTC", false),
+ )
+ .unwrap();
+ if let ColumnarValue::Array(cast_array) = cast_array {
+ assert_eq!(5, cast_array.len());
+ let a = cast_array.as_struct().column(0).as_string::();
+ assert_eq!("1", a.value(0));
+ } else {
+ unreachable!()
+ }
+ }
+
+ #[test]
+ fn test_cast_struct_to_struct_drop_column() {
+ let a: ArrayRef = Arc::new(Int32Array::from(vec![
+ Some(1),
+ Some(2),
+ None,
+ Some(4),
+ Some(5),
+ ]));
+ let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"]));
+ let c: ArrayRef = Arc::new(StructArray::from(vec![
+ (Arc::new(Field::new("a", DataType::Int32, true)), a),
+ (Arc::new(Field::new("b", DataType::Utf8, true)), b),
+ ]));
+ // change type of "a" from Int32 to Utf8 and drop "b"
+ let fields = Fields::from(vec![Field::new("a", DataType::Utf8, true)]);
+ let cast_array = spark_parquet_convert(
+ ColumnarValue::Array(c),
+ &DataType::Struct(fields),
+ &SparkParquetOptions::new(EvalMode::Legacy, "UTC", false),
+ )
+ .unwrap();
+ if let ColumnarValue::Array(cast_array) = cast_array {
+ assert_eq!(5, cast_array.len());
+ let struct_array = cast_array.as_struct();
+ assert_eq!(1, struct_array.columns().len());
+ let a = struct_array.column(0).as_string::();
+ assert_eq!("1", a.value(0));
+ } else {
+ unreachable!()
+ }
+ }
+}
diff --git a/native/core/src/parquet/schema_adapter.rs b/native/core/src/parquet/schema_adapter.rs
new file mode 100644
index 0000000000..1eee68c95b
--- /dev/null
+++ b/native/core/src/parquet/schema_adapter.rs
@@ -0,0 +1,626 @@
+// 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.
+
+//! Custom schema adapter that uses Spark-compatible conversions
+
+use crate::parquet::parquet_support::{spark_parquet_convert, SparkParquetOptions};
+use arrow_array::{new_null_array, Array, RecordBatch, RecordBatchOptions};
+use arrow_schema::{DataType, Schema, SchemaRef};
+use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper};
+use datafusion_comet_spark_expr::EvalMode;
+use datafusion_common::plan_err;
+use datafusion_expr::ColumnarValue;
+use std::collections::HashMap;
+use std::sync::Arc;
+
+/// An implementation of DataFusion's `SchemaAdapterFactory` that uses a Spark-compatible
+/// `cast` implementation.
+#[derive(Clone, Debug)]
+pub struct SparkSchemaAdapterFactory {
+ /// Spark cast options
+ parquet_options: SparkParquetOptions,
+}
+
+impl SparkSchemaAdapterFactory {
+ pub fn new(options: SparkParquetOptions) -> Self {
+ Self {
+ parquet_options: options,
+ }
+ }
+}
+
+impl SchemaAdapterFactory for SparkSchemaAdapterFactory {
+ /// Create a new factory for mapping batches from a file schema to a table
+ /// schema.
+ ///
+ /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with
+ /// the same schema for both the projected table schema and the table
+ /// schema.
+ fn create(
+ &self,
+ required_schema: SchemaRef,
+ table_schema: SchemaRef,
+ ) -> Box {
+ Box::new(SparkSchemaAdapter {
+ required_schema,
+ table_schema,
+ parquet_options: self.parquet_options.clone(),
+ })
+ }
+}
+
+/// This SchemaAdapter requires both the table schema and the projected table
+/// schema. See [`SchemaMapping`] for more details
+#[derive(Clone, Debug)]
+pub struct SparkSchemaAdapter {
+ /// The schema for the table, projected to include only the fields being output (projected) by the
+ /// associated ParquetExec
+ required_schema: SchemaRef,
+ /// The entire table schema for the table we're using this to adapt.
+ ///
+ /// This is used to evaluate any filters pushed down into the scan
+ /// which may refer to columns that are not referred to anywhere
+ /// else in the plan.
+ table_schema: SchemaRef,
+ /// Spark cast options
+ parquet_options: SparkParquetOptions,
+}
+
+impl SchemaAdapter for SparkSchemaAdapter {
+ /// Map a column index in the table schema to a column index in a particular
+ /// file schema
+ ///
+ /// Panics if index is not in range for the table schema
+ fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option {
+ let field = self.required_schema.field(index);
+ Some(file_schema.fields.find(field.name())?.0)
+ }
+
+ /// Creates a `SchemaMapping` for casting or mapping the columns from the
+ /// file schema to the table schema.
+ ///
+ /// If the provided `file_schema` contains columns of a different type to
+ /// the expected `table_schema`, the method will attempt to cast the array
+ /// data from the file schema to the table schema where possible.
+ ///
+ /// Returns a [`SchemaMapping`] that can be applied to the output batch
+ /// along with an ordered list of columns to project from the file
+ fn map_schema(
+ &self,
+ file_schema: &Schema,
+ ) -> datafusion_common::Result<(Arc, Vec)> {
+ let mut projection = Vec::with_capacity(file_schema.fields().len());
+ let mut field_mappings = vec![None; self.required_schema.fields().len()];
+
+ for (file_idx, file_field) in file_schema.fields.iter().enumerate() {
+ if let Some((table_idx, table_field)) =
+ self.required_schema.fields().find(file_field.name())
+ {
+ if cast_supported(
+ file_field.data_type(),
+ table_field.data_type(),
+ &self.parquet_options,
+ ) {
+ field_mappings[table_idx] = Some(projection.len());
+ projection.push(file_idx);
+ } else {
+ return plan_err!(
+ "Cannot cast file schema field {} of type {:?} to required schema field of type {:?}",
+ file_field.name(),
+ file_field.data_type(),
+ table_field.data_type()
+ );
+ }
+ }
+ }
+
+ Ok((
+ Arc::new(SchemaMapping {
+ required_schema: Arc::::clone(&self.required_schema),
+ field_mappings,
+ table_schema: Arc::::clone(&self.table_schema),
+ parquet_options: self.parquet_options.clone(),
+ }),
+ projection,
+ ))
+ }
+}
+
+// TODO SchemaMapping is mostly copied from DataFusion but calls spark_cast
+// instead of arrow cast - can we reduce the amount of code copied here and make
+// the DataFusion version more extensible?
+
+/// The SchemaMapping struct holds a mapping from the file schema to the table
+/// schema and any necessary type conversions.
+///
+/// Note, because `map_batch` and `map_partial_batch` functions have different
+/// needs, this struct holds two schemas:
+///
+/// 1. The projected **table** schema
+/// 2. The full table schema
+///
+/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which
+/// has the projected schema, since that's the schema which is supposed to come
+/// out of the execution of this query. Thus `map_batch` uses
+/// `projected_table_schema` as it can only operate on the projected fields.
+///
+/// [`map_partial_batch`] is used to create a RecordBatch with a schema that
+/// can be used for Parquet predicate pushdown, meaning that it may contain
+/// fields which are not in the projected schema (as the fields that parquet
+/// pushdown filters operate can be completely distinct from the fields that are
+/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses
+/// `table_schema` to create the resulting RecordBatch (as it could be operating
+/// on any fields in the schema).
+///
+/// [`map_batch`]: Self::map_batch
+/// [`map_partial_batch`]: Self::map_partial_batch
+#[derive(Debug)]
+pub struct SchemaMapping {
+ /// The schema of the table. This is the expected schema after conversion
+ /// and it should match the schema of the query result.
+ required_schema: SchemaRef,
+ /// Mapping from field index in `projected_table_schema` to index in
+ /// projected file_schema.
+ ///
+ /// They are Options instead of just plain `usize`s because the table could
+ /// have fields that don't exist in the file.
+ field_mappings: Vec>,
+ /// The entire table schema, as opposed to the projected_table_schema (which
+ /// only contains the columns that we are projecting out of this query).
+ /// This contains all fields in the table, regardless of if they will be
+ /// projected out or not.
+ table_schema: SchemaRef,
+ /// Spark cast options
+ parquet_options: SparkParquetOptions,
+}
+
+impl SchemaMapper for SchemaMapping {
+ /// Adapts a `RecordBatch` to match the `projected_table_schema` using the stored mapping and
+ /// conversions. The produced RecordBatch has a schema that contains only the projected
+ /// columns, so if one needs a RecordBatch with a schema that references columns which are not
+ /// in the projected, it would be better to use `map_partial_batch`
+ fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result {
+ let batch_rows = batch.num_rows();
+ let batch_cols = batch.columns().to_vec();
+
+ let cols = self
+ .required_schema
+ // go through each field in the projected schema
+ .fields()
+ .iter()
+ // and zip it with the index that maps fields from the projected table schema to the
+ // projected file schema in `batch`
+ .zip(&self.field_mappings)
+ // and for each one...
+ .map(|(field, file_idx)| {
+ file_idx.map_or_else(
+ // If this field only exists in the table, and not in the file, then we know
+ // that it's null, so just return that.
+ || Ok(new_null_array(field.data_type(), batch_rows)),
+ // However, if it does exist in both, then try to cast it to the correct output
+ // type
+ |batch_idx| {
+ spark_parquet_convert(
+ ColumnarValue::Array(Arc::clone(&batch_cols[batch_idx])),
+ field.data_type(),
+ &self.parquet_options,
+ )?
+ .into_array(batch_rows)
+ },
+ )
+ })
+ .collect::, _>>()?;
+
+ // Necessary to handle empty batches
+ let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows()));
+
+ let schema = Arc::::clone(&self.required_schema);
+ let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?;
+ Ok(record_batch)
+ }
+
+ /// Adapts a [`RecordBatch`]'s schema into one that has all the correct output types and only
+ /// contains the fields that exist in both the file schema and table schema.
+ ///
+ /// Unlike `map_batch` this method also preserves the columns that
+ /// may not appear in the final output (`projected_table_schema`) but may
+ /// appear in push down predicates
+ fn map_partial_batch(&self, batch: RecordBatch) -> datafusion_common::Result {
+ let batch_cols = batch.columns().to_vec();
+ let schema = batch.schema();
+
+ // for each field in the batch's schema (which is based on a file, not a table)...
+ let (cols, fields) = schema
+ .fields()
+ .iter()
+ .zip(batch_cols.iter())
+ .flat_map(|(field, batch_col)| {
+ self.table_schema
+ // try to get the same field from the table schema that we have stored in self
+ .field_with_name(field.name())
+ // and if we don't have it, that's fine, ignore it. This may occur when we've
+ // created an external table whose fields are a subset of the fields in this
+ // file, then tried to read data from the file into this table. If that is the
+ // case here, it's fine to ignore because we don't care about this field
+ // anyways
+ .ok()
+ // but if we do have it,
+ .map(|table_field| {
+ // try to cast it into the correct output type. we don't want to ignore this
+ // error, though, so it's propagated.
+ spark_parquet_convert(
+ ColumnarValue::Array(Arc::clone(batch_col)),
+ table_field.data_type(),
+ &self.parquet_options,
+ )?
+ .into_array(batch_col.len())
+ // and if that works, return the field and column.
+ .map(|new_col| (new_col, table_field.clone()))
+ })
+ })
+ .collect::, _>>()?
+ .into_iter()
+ .unzip::<_, _, Vec<_>, Vec<_>>();
+
+ // Necessary to handle empty batches
+ let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows()));
+
+ let schema = Arc::new(Schema::new_with_metadata(fields, schema.metadata().clone()));
+ let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?;
+ Ok(record_batch)
+ }
+}
+
+/// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account.
+fn cast_supported(from_type: &DataType, to_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+
+ let from_type = if let Dictionary(_, dt) = from_type {
+ dt
+ } else {
+ from_type
+ };
+
+ let to_type = if let Dictionary(_, dt) = to_type {
+ dt
+ } else {
+ to_type
+ };
+
+ if from_type == to_type {
+ return true;
+ }
+
+ match (from_type, to_type) {
+ (Boolean, _) => can_convert_from_boolean(to_type, options),
+ (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64)
+ if options.allow_cast_unsigned_ints =>
+ {
+ true
+ }
+ (Int8, _) => can_convert_from_byte(to_type, options),
+ (Int16, _) => can_convert_from_short(to_type, options),
+ (Int32, _) => can_convert_from_int(to_type, options),
+ (Int64, _) => can_convert_from_long(to_type, options),
+ (Float32, _) => can_convert_from_float(to_type, options),
+ (Float64, _) => can_convert_from_double(to_type, options),
+ (Decimal128(p, s), _) => can_convert_from_decimal(p, s, to_type, options),
+ (Timestamp(_, None), _) => can_convert_from_timestamp_ntz(to_type, options),
+ (Timestamp(_, Some(_)), _) => can_convert_from_timestamp(to_type, options),
+ (Utf8 | LargeUtf8, _) => can_convert_from_string(to_type, options),
+ (_, Utf8 | LargeUtf8) => can_cast_to_string(from_type, options),
+ (Struct(from_fields), Struct(to_fields)) => {
+ // TODO some of this logic may be specific to converting Parquet to Spark
+ let mut field_types = HashMap::new();
+ for field in from_fields {
+ field_types.insert(field.name(), field.data_type());
+ }
+ if field_types.iter().len() != from_fields.len() {
+ return false;
+ }
+ for field in to_fields {
+ if let Some(from_type) = field_types.get(&field.name()) {
+ if !cast_supported(from_type, field.data_type(), options) {
+ return false;
+ }
+ } else {
+ return false;
+ }
+ }
+ true
+ }
+ _ => false,
+ }
+}
+
+fn can_convert_from_string(to_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match to_type {
+ Boolean | Int8 | Int16 | Int32 | Int64 | Binary => true,
+ Float32 | Float64 => {
+ // https://github.com/apache/datafusion-comet/issues/326
+ // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'.
+ // Does not support ANSI mode.
+ options.allow_incompat
+ }
+ Decimal128(_, _) => {
+ // https://github.com/apache/datafusion-comet/issues/325
+ // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'.
+ // Does not support ANSI mode. Returns 0.0 instead of null if input contains no digits
+
+ options.allow_incompat
+ }
+ Date32 | Date64 => {
+ // https://github.com/apache/datafusion-comet/issues/327
+ // Only supports years between 262143 BC and 262142 AD
+ options.allow_incompat
+ }
+ Timestamp(_, _) if options.eval_mode == EvalMode::Ansi => {
+ // ANSI mode not supported
+ false
+ }
+ Timestamp(_, Some(tz)) if tz.as_ref() != "UTC" => {
+ // Cast will use UTC instead of $timeZoneId
+ options.allow_incompat
+ }
+ Timestamp(_, _) => {
+ // https://github.com/apache/datafusion-comet/issues/328
+ // Not all valid formats are supported
+ options.allow_incompat
+ }
+ _ => false,
+ }
+}
+
+fn can_cast_to_string(from_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match from_type {
+ Boolean | Int8 | Int16 | Int32 | Int64 | Date32 | Date64 | Timestamp(_, _) => true,
+ Float32 | Float64 => {
+ // There can be differences in precision.
+ // For example, the input \"1.4E-45\" will produce 1.0E-45 " +
+ // instead of 1.4E-45"))
+ true
+ }
+ Decimal128(_, _) => {
+ // https://github.com/apache/datafusion-comet/issues/1068
+ // There can be formatting differences in some case due to Spark using
+ // scientific notation where Comet does not
+ true
+ }
+ Binary => {
+ // https://github.com/apache/datafusion-comet/issues/377
+ // Only works for binary data representing valid UTF-8 strings
+ options.allow_incompat
+ }
+ Struct(fields) => fields
+ .iter()
+ .all(|f| can_cast_to_string(f.data_type(), options)),
+ _ => false,
+ }
+}
+
+fn can_convert_from_timestamp_ntz(to_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match to_type {
+ Timestamp(_, _) | Date32 | Date64 | Utf8 => {
+ // incompatible
+ options.allow_incompat
+ }
+ _ => {
+ // unsupported
+ false
+ }
+ }
+}
+
+fn can_convert_from_timestamp(to_type: &DataType, _options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match to_type {
+ Timestamp(_, _) => true,
+ Boolean | Int8 | Int16 => {
+ // https://github.com/apache/datafusion-comet/issues/352
+ // this seems like an edge case that isn't important for us to support
+ false
+ }
+ Int64 => {
+ // https://github.com/apache/datafusion-comet/issues/352
+ true
+ }
+ Date32 | Date64 | Utf8 | Decimal128(_, _) => true,
+ _ => {
+ // unsupported
+ false
+ }
+ }
+}
+
+fn can_convert_from_boolean(to_type: &DataType, _: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ matches!(to_type, Int8 | Int16 | Int32 | Int64 | Float32 | Float64)
+}
+
+fn can_convert_from_byte(to_type: &DataType, _: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ matches!(
+ to_type,
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _)
+ )
+}
+
+fn can_convert_from_short(to_type: &DataType, _: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ matches!(
+ to_type,
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _)
+ )
+}
+
+fn can_convert_from_int(to_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match to_type {
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Utf8 => true,
+ Decimal128(_, _) => {
+ // incompatible: no overflow check
+ options.allow_incompat
+ }
+ _ => false,
+ }
+}
+
+fn can_convert_from_long(to_type: &DataType, options: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ match to_type {
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true,
+ Decimal128(_, _) => {
+ // incompatible: no overflow check
+ options.allow_incompat
+ }
+ _ => false,
+ }
+}
+
+fn can_convert_from_float(to_type: &DataType, _: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ matches!(
+ to_type,
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float64 | Decimal128(_, _)
+ )
+}
+
+fn can_convert_from_double(to_type: &DataType, _: &SparkParquetOptions) -> bool {
+ use DataType::*;
+ matches!(
+ to_type,
+ Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Decimal128(_, _)
+ )
+}
+
+fn can_convert_from_decimal(
+ p1: &u8,
+ _s1: &i8,
+ to_type: &DataType,
+ options: &SparkParquetOptions,
+) -> bool {
+ use DataType::*;
+ match to_type {
+ Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true,
+ Decimal128(p2, _) => {
+ if p2 < p1 {
+ // https://github.com/apache/datafusion/issues/13492
+ // Incompatible(Some("Casting to smaller precision is not supported"))
+ options.allow_incompat
+ } else {
+ true
+ }
+ }
+ _ => false,
+ }
+}
+
+#[cfg(test)]
+mod test {
+ use crate::parquet::parquet_support::SparkParquetOptions;
+ use crate::parquet::schema_adapter::SparkSchemaAdapterFactory;
+ use arrow::array::{Int32Array, StringArray};
+ use arrow::datatypes::{DataType, Field, Schema};
+ use arrow::record_batch::RecordBatch;
+ use arrow_array::UInt32Array;
+ use arrow_schema::SchemaRef;
+ use datafusion::datasource::listing::PartitionedFile;
+ use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec};
+ use datafusion::execution::object_store::ObjectStoreUrl;
+ use datafusion::execution::TaskContext;
+ use datafusion::physical_plan::ExecutionPlan;
+ use datafusion_comet_spark_expr::test_common::file_util::get_temp_filename;
+ use datafusion_comet_spark_expr::EvalMode;
+ use datafusion_common::DataFusionError;
+ use futures::StreamExt;
+ use parquet::arrow::ArrowWriter;
+ use std::fs::File;
+ use std::sync::Arc;
+
+ #[tokio::test]
+ async fn parquet_roundtrip_int_as_string() -> Result<(), DataFusionError> {
+ let file_schema = Arc::new(Schema::new(vec![
+ Field::new("id", DataType::Int32, false),
+ Field::new("name", DataType::Utf8, false),
+ ]));
+
+ let ids = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc;
+ let names = Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"]))
+ as Arc;
+ let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids, names])?;
+
+ let required_schema = Arc::new(Schema::new(vec![
+ Field::new("id", DataType::Utf8, false),
+ Field::new("name", DataType::Utf8, false),
+ ]));
+
+ let _ = roundtrip(&batch, required_schema).await?;
+
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn parquet_roundtrip_unsigned_int() -> Result<(), DataFusionError> {
+ let file_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::UInt32, false)]));
+
+ let ids = Arc::new(UInt32Array::from(vec![1, 2, 3])) as Arc;
+ let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids])?;
+
+ let required_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)]));
+
+ let _ = roundtrip(&batch, required_schema).await?;
+
+ Ok(())
+ }
+
+ /// Create a Parquet file containing a single batch and then read the batch back using
+ /// the specified required_schema. This will cause the SchemaAdapter code to be used.
+ async fn roundtrip(
+ batch: &RecordBatch,
+ required_schema: SchemaRef,
+ ) -> Result {
+ let filename = get_temp_filename();
+ let filename = filename.as_path().as_os_str().to_str().unwrap().to_string();
+ let file = File::create(&filename)?;
+ let mut writer = ArrowWriter::try_new(file, Arc::clone(&batch.schema()), None)?;
+ writer.write(batch)?;
+ writer.close()?;
+
+ let object_store_url = ObjectStoreUrl::local_filesystem();
+ let file_scan_config = FileScanConfig::new(object_store_url, required_schema)
+ .with_file_groups(vec![vec![PartitionedFile::from_path(
+ filename.to_string(),
+ )?]]);
+
+ let mut spark_parquet_options = SparkParquetOptions::new(EvalMode::Legacy, "UTC", false);
+ spark_parquet_options.allow_cast_unsigned_ints = true;
+
+ let parquet_exec = ParquetExec::builder(file_scan_config)
+ .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new(
+ spark_parquet_options,
+ )))
+ .build();
+
+ let mut stream = parquet_exec
+ .execute(0, Arc::new(TaskContext::default()))
+ .unwrap();
+ stream.next().await.unwrap()
+ }
+}
diff --git a/native/core/src/parquet/util/jni.rs b/native/core/src/parquet/util/jni.rs
index b61fbeab32..d9fa49922a 100644
--- a/native/core/src/parquet/util/jni.rs
+++ b/native/core/src/parquet/util/jni.rs
@@ -24,11 +24,16 @@ use jni::{
JNIEnv,
};
+use arrow::error::ArrowError;
+use arrow::ipc::reader::StreamReader;
+use datafusion_execution::object_store::ObjectStoreUrl;
+use object_store::path::Path;
use parquet::{
basic::{Encoding, LogicalType, TimeUnit, Type as PhysicalType},
format::{MicroSeconds, MilliSeconds, NanoSeconds},
schema::types::{ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder},
};
+use url::{ParseError, Url};
/// Convert primitives from Spark side into a `ColumnDescriptor`.
#[allow(clippy::too_many_arguments)]
@@ -198,3 +203,52 @@ fn fix_type_length(t: &PhysicalType, type_length: i32) -> i32 {
_ => type_length,
}
}
+
+pub fn deserialize_schema(ipc_bytes: &[u8]) -> Result {
+ let reader = StreamReader::try_new(std::io::Cursor::new(ipc_bytes), None)?;
+ let schema = reader.schema().as_ref().clone();
+ Ok(schema)
+}
+
+// parses the url and returns a tuple of the scheme and object store path
+pub fn get_file_path(url_: String) -> Result<(ObjectStoreUrl, Path), ParseError> {
+ // we define origin of a url as scheme + "://" + authority + ["/" + bucket]
+ let url = Url::parse(url_.as_ref()).unwrap();
+ let mut object_store_origin = url.scheme().to_owned();
+ let mut object_store_path = Path::from_url_path(url.path()).unwrap();
+ if object_store_origin == "s3a" {
+ object_store_origin = "s3".to_string();
+ object_store_origin.push_str("://");
+ object_store_origin.push_str(url.authority());
+ object_store_origin.push('/');
+ let path_splits = url.path_segments().map(|c| c.collect::>()).unwrap();
+ object_store_origin.push_str(path_splits.first().unwrap());
+ let new_path = path_splits[1..path_splits.len() - 1].join("/");
+ //TODO: (ARROW NATIVE) check the use of unwrap here
+ object_store_path = Path::from_url_path(new_path.clone().as_str()).unwrap();
+ } else {
+ object_store_origin.push_str("://");
+ object_store_origin.push_str(url.authority());
+ object_store_origin.push('/');
+ }
+ Ok((
+ ObjectStoreUrl::parse(object_store_origin).unwrap(),
+ object_store_path,
+ ))
+}
+
+#[cfg(test)]
+mod tests {
+ use super::*;
+
+ #[test]
+ fn test_get_file_path() {
+ let inp = "file:///comet/spark-warehouse/t1/part1=2019-01-01%2011%253A11%253A11/part-00000-84d7ed74-8f28-456c-9270-f45376eea144.c000.snappy.parquet";
+ let expected = "comet/spark-warehouse/t1/part1=2019-01-01 11%3A11%3A11/part-00000-84d7ed74-8f28-456c-9270-f45376eea144.c000.snappy.parquet";
+
+ if let Ok((_obj_store_url, path)) = get_file_path(inp.to_string()) {
+ let actual = path.to_string();
+ assert_eq!(actual, expected);
+ }
+ }
+}
diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto
index c6f310d65c..7e5aa975fb 100644
--- a/native/proto/src/proto/operator.proto
+++ b/native/proto/src/proto/operator.proto
@@ -46,9 +46,30 @@ message Operator {
SortMergeJoin sort_merge_join = 108;
HashJoin hash_join = 109;
Window window = 110;
+ NativeScan native_scan = 111;
}
}
+message SparkPartitionedFile {
+ string file_path = 1;
+ int64 start = 2;
+ int64 length = 3;
+ int64 file_size = 4;
+ repeated spark.spark_expression.Expr partition_values = 5;
+}
+
+// This name and the one above are not great, but they correspond to the (unfortunate) Spark names.
+// I prepended "Spark" since I think there's a name collision on the native side, but we can revisit.
+message SparkFilePartition {
+ repeated SparkPartitionedFile partitioned_file = 1;
+}
+
+message SparkStructField {
+ string name = 1;
+ spark.spark_expression.DataType data_type = 2;
+ bool nullable = 3;
+}
+
message Scan {
repeated spark.spark_expression.DataType fields = 1;
// The source of the scan (e.g. file scan, broadcast exchange, shuffle, etc). This
@@ -57,6 +78,21 @@ message Scan {
string source = 2;
}
+message NativeScan {
+ 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
+ // debug mode.
+ string source = 2;
+ repeated SparkStructField required_schema = 3;
+ 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;
+}
+
message Projection {
repeated spark.spark_expression.Expr project_list = 1;
}
diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs
index 6e0e0915cd..0f59761325 100644
--- a/native/spark-expr/src/cast.rs
+++ b/native/spark-expr/src/cast.rs
@@ -37,7 +37,7 @@ use arrow::{
};
use arrow_array::builder::StringBuilder;
use arrow_array::{DictionaryArray, StringArray, StructArray};
-use arrow_schema::{DataType, Field, Schema};
+use arrow_schema::{DataType, Schema};
use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike};
use datafusion_common::{
cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue,
@@ -49,6 +49,7 @@ use num::{
ToPrimitive,
};
use regex::Regex;
+use std::collections::HashMap;
use std::str::FromStr;
use std::{
any::Any,
@@ -803,11 +804,15 @@ pub struct SparkCastOptions {
pub eval_mode: EvalMode,
/// When cast from/to timezone related types, we need timezone, which will be resolved with
/// session local timezone by an analyzer in Spark.
+ // TODO we should change timezone to Tz to avoid repeated parsing
pub timezone: String,
/// Allow casts that are supported but not guaranteed to be 100% compatible
pub allow_incompat: bool,
/// Support casting unsigned ints to signed ints (used by Parquet SchemaAdapter)
pub allow_cast_unsigned_ints: bool,
+ /// We also use the cast logic for adapting Parquet schemas, so this flag is used
+ /// for that use case
+ pub is_adapting_schema: bool,
}
impl SparkCastOptions {
@@ -817,6 +822,7 @@ impl SparkCastOptions {
timezone: timezone.to_string(),
allow_incompat,
allow_cast_unsigned_ints: false,
+ is_adapting_schema: false,
}
}
@@ -826,6 +832,7 @@ impl SparkCastOptions {
timezone: "".to_string(),
allow_incompat,
allow_cast_unsigned_ints: false,
+ is_adapting_schema: false,
}
}
}
@@ -952,7 +959,9 @@ fn cast_array(
{
Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?)
}
- _ if is_datafusion_spark_compatible(from_type, to_type, cast_options.allow_incompat) => {
+ _ if cast_options.is_adapting_schema
+ || is_datafusion_spark_compatible(from_type, to_type, cast_options.allow_incompat) =>
+ {
// use DataFusion cast only when we know that it is compatible with Spark
Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?)
}
@@ -1058,17 +1067,28 @@ fn cast_struct_to_struct(
cast_options: &SparkCastOptions,
) -> DataFusionResult {
match (from_type, to_type) {
- (DataType::Struct(_), DataType::Struct(to_fields)) => {
- let mut cast_fields: Vec<(Arc, ArrayRef)> = Vec::with_capacity(to_fields.len());
+ (DataType::Struct(from_fields), DataType::Struct(to_fields)) => {
+ // TODO some of this logic may be specific to converting Parquet to Spark
+ let mut field_name_to_index_map = HashMap::new();
+ for (i, field) in from_fields.iter().enumerate() {
+ field_name_to_index_map.insert(field.name(), i);
+ }
+ assert_eq!(field_name_to_index_map.len(), from_fields.len());
+ let mut cast_fields: Vec = Vec::with_capacity(to_fields.len());
for i in 0..to_fields.len() {
+ let from_index = field_name_to_index_map[to_fields[i].name()];
let cast_field = cast_array(
- Arc::clone(array.column(i)),
+ Arc::clone(array.column(from_index)),
to_fields[i].data_type(),
cast_options,
)?;
- cast_fields.push((Arc::clone(&to_fields[i]), cast_field));
+ cast_fields.push(cast_field);
}
- Ok(Arc::new(StructArray::from(cast_fields)))
+ Ok(Arc::new(StructArray::new(
+ to_fields.clone(),
+ cast_fields,
+ array.nulls().cloned(),
+ )))
}
_ => unreachable!(),
}
diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs
index 18a2314fb1..ed04941bdb 100644
--- a/native/spark-expr/src/utils.rs
+++ b/native/spark-expr/src/utils.rs
@@ -19,7 +19,7 @@ use arrow_array::{
cast::as_primitive_array,
types::{Int32Type, TimestampMicrosecondType},
};
-use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION};
+use arrow_schema::{ArrowError, DataType, TimeUnit, DECIMAL128_MAX_PRECISION};
use std::sync::Arc;
use crate::timezone::Tz;
@@ -27,6 +27,7 @@ use arrow::{
array::{as_dictionary_array, Array, ArrayRef, PrimitiveArray},
temporal_conversions::as_datetime,
};
+use arrow_array::types::TimestampMillisecondType;
use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION};
use chrono::{DateTime, Offset, TimeZone};
@@ -71,6 +72,9 @@ pub fn array_with_timezone(
Some(DataType::Timestamp(_, Some(_))) => {
timestamp_ntz_to_timestamp(array, timezone.as_str(), Some(timezone.as_str()))
}
+ Some(DataType::Timestamp(_, None)) => {
+ timestamp_ntz_to_timestamp(array, timezone.as_str(), None)
+ }
_ => {
// Not supported
panic!(
@@ -81,7 +85,7 @@ pub fn array_with_timezone(
}
}
}
- DataType::Timestamp(_, Some(_)) => {
+ DataType::Timestamp(TimeUnit::Microsecond, Some(_)) => {
assert!(!timezone.is_empty());
let array = as_primitive_array::(&array);
let array_with_timezone = array.clone().with_timezone(timezone.clone());
@@ -93,6 +97,18 @@ pub fn array_with_timezone(
_ => Ok(array),
}
}
+ DataType::Timestamp(TimeUnit::Millisecond, Some(_)) => {
+ assert!(!timezone.is_empty());
+ let array = as_primitive_array::(&array);
+ let array_with_timezone = array.clone().with_timezone(timezone.clone());
+ let array = Arc::new(array_with_timezone) as ArrayRef;
+ match to_type {
+ Some(DataType::Utf8) | Some(DataType::Date32) => {
+ pre_timestamp_cast(array, timezone)
+ }
+ _ => Ok(array),
+ }
+ }
DataType::Dictionary(_, value_type)
if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) =>
{
@@ -128,7 +144,7 @@ fn timestamp_ntz_to_timestamp(
) -> Result {
assert!(!tz.is_empty());
match array.data_type() {
- DataType::Timestamp(_, None) => {
+ DataType::Timestamp(TimeUnit::Microsecond, None) => {
let array = as_primitive_array::(&array);
let tz: Tz = tz.parse()?;
let array: PrimitiveArray = array.try_unary(|value| {
@@ -147,6 +163,25 @@ fn timestamp_ntz_to_timestamp(
};
Ok(Arc::new(array_with_tz))
}
+ DataType::Timestamp(TimeUnit::Millisecond, None) => {
+ let array = as_primitive_array::(&array);
+ let tz: Tz = tz.parse()?;
+ let array: PrimitiveArray = array.try_unary(|value| {
+ as_datetime::(value)
+ .ok_or_else(|| datetime_cast_err(value))
+ .map(|local_datetime| {
+ let datetime: DateTime =
+ tz.from_local_datetime(&local_datetime).unwrap();
+ datetime.timestamp_millis()
+ })
+ })?;
+ let array_with_tz = if let Some(to_tz) = to_timezone {
+ array.with_timezone(to_tz)
+ } else {
+ array
+ };
+ Ok(Arc::new(array_with_tz))
+ }
_ => Ok(array),
}
}
diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala
index 6066c890d3..7a70181401 100644
--- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala
+++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala
@@ -70,6 +70,7 @@ class CometExecIterator(
id,
cometBatchIterators,
protobufQueryPlan,
+ numParts,
nativeMetrics,
new CometTaskMemoryManager(id),
batchSize = COMET_BATCH_SIZE.get(),
diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
index e786eece22..c9d8ce55b1 100644
--- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
+++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
@@ -53,7 +53,7 @@ import org.apache.spark.sql.types.{DoubleType, FloatType}
import org.apache.comet.CometConf._
import org.apache.comet.CometExplainInfo.getActualPlan
-import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos}
+import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isOffHeapEnabled, isSpark34Plus, isSpark40Plus, isTesting, shouldApplySparkToColumnar, withInfo, withInfos}
import org.apache.comet.parquet.{CometParquetScan, SupportsComet}
import org.apache.comet.rules.RewriteJoin
import org.apache.comet.serde.OperatorOuterClass.Operator
@@ -188,6 +188,28 @@ class CometSparkSessionExtensions
scanExec
}
+ // data source V1
+ case scanExec @ FileSourceScanExec(
+ HadoopFsRelation(_, partitionSchema, _, _, _: ParquetFileFormat, _),
+ _: Seq[_],
+ requiredSchema,
+ _,
+ _,
+ _,
+ _,
+ _,
+ _)
+ if CometNativeScanExec.isSchemaSupported(requiredSchema)
+ && CometNativeScanExec.isSchemaSupported(partitionSchema)
+ // TODO we only enable full native scan if COMET_EXEC_ENABLED is enabled
+ // but this is not really what we want .. we currently insert `CometScanExec`
+ // here and then it gets replaced with `CometNativeScanExec` in `CometExecRule`
+ // but that only happens if `COMET_EXEC_ENABLED` is enabled
+ && COMET_EXEC_ENABLED.get()
+ && COMET_NATIVE_SCAN_IMPL.get() == CometConf.SCAN_NATIVE_DATAFUSION =>
+ logInfo("Comet extension enabled for v1 full native Scan")
+ CometScanExec(scanExec, session)
+
// data source V1
case scanExec @ FileSourceScanExec(
HadoopFsRelation(_, partitionSchema, _, _, fileFormat, _),
@@ -353,6 +375,13 @@ class CometSparkSessionExtensions
}
plan.transformUp {
+ // Fully native scan for V1
+ case scan: CometScanExec
+ if COMET_NATIVE_SCAN_IMPL.get().equals(CometConf.SCAN_NATIVE_DATAFUSION) =>
+ val nativeOp = QueryPlanSerde.operator2Proto(scan).get
+ CometNativeScanExec(nativeOp, scan.wrapped, scan.session)
+
+ // Comet JVM + native scan for V1 and V2
case op if isCometScan(op) =>
val nativeOp = QueryPlanSerde.operator2Proto(op).get
CometScanWrapper(nativeOp, op)
@@ -944,6 +973,14 @@ class CometSparkSessionExtensions
}
override def apply(plan: SparkPlan): SparkPlan = {
+
+ // Comet required off-heap memory to be enabled
+ if (!isOffHeapEnabled(conf) && !isTesting) {
+ logWarning("Comet native exec disabled because spark.memory.offHeap.enabled=false")
+ withInfo(plan, "Comet native exec disabled because spark.memory.offHeap.enabled=false")
+ return plan
+ }
+
// DataFusion doesn't have ANSI mode. For now we just disable CometExec if ANSI mode is
// enabled.
if (isANSIEnabled(conf)) {
@@ -1204,12 +1241,21 @@ object CometSparkSessionExtensions extends Logging {
}
}
+ private[comet] def isOffHeapEnabled(conf: SQLConf): Boolean =
+ conf.getConfString("spark.memory.offHeap.enabled", "false").toBoolean
+
+ // Copied from org.apache.spark.util.Utils which is private to Spark.
+ private[comet] def isTesting: Boolean = {
+ System.getenv("SPARK_TESTING") != null || System.getProperty("spark.testing") != null
+ }
+
// Check whether Comet shuffle is enabled:
// 1. `COMET_EXEC_SHUFFLE_ENABLED` is true
// 2. `spark.shuffle.manager` is set to `CometShuffleManager`
// 3. Off-heap memory is enabled || Spark/Comet unit testing
private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean =
- COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf)
+ COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) &&
+ (isOffHeapEnabled(conf) || isTesting)
private[comet] def getCometShuffleNotEnabledReason(conf: SQLConf): Option[String] = {
if (!COMET_EXEC_SHUFFLE_ENABLED.get(conf)) {
diff --git a/spark/src/main/scala/org/apache/comet/DataTypeSupport.scala b/spark/src/main/scala/org/apache/comet/DataTypeSupport.scala
index c49a2c465c..5c765003cf 100644
--- a/spark/src/main/scala/org/apache/comet/DataTypeSupport.scala
+++ b/spark/src/main/scala/org/apache/comet/DataTypeSupport.scala
@@ -38,7 +38,9 @@ trait DataTypeSupport {
case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType |
BinaryType | StringType | _: DecimalType | DateType | TimestampType =>
true
- case t: DataType if t.typeName == "timestamp_ntz" => true
+ case t: DataType if t.typeName == "timestamp_ntz" =>
+ true
+ true
case _ => false
}
diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala
index 7a8c061a25..c293905c96 100644
--- a/spark/src/main/scala/org/apache/comet/Native.scala
+++ b/spark/src/main/scala/org/apache/comet/Native.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.comet.CometMetricNode
class Native extends NativeBase {
+ // scalastyle:off
/**
* Create a native query plan from execution SparkPlan serialized in bytes.
* @param id
@@ -50,6 +51,7 @@ class Native extends NativeBase {
id: Long,
iterators: Array[CometBatchIterator],
plan: Array[Byte],
+ partitionCount: Int,
metrics: CometMetricNode,
taskMemoryManager: CometTaskMemoryManager,
batchSize: Int,
diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala
index 52d8d09a0d..b6a511a5b7 100644
--- a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala
+++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala
@@ -100,6 +100,8 @@ class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with
// Comet specific configurations
val capacity = CometConf.COMET_BATCH_SIZE.get(sqlConf)
+ val nativeRecordBatchReaderEnabled =
+ CometConf.COMET_NATIVE_SCAN_IMPL.get(sqlConf).equals(CometConf.SCAN_NATIVE_ICEBERG_COMPAT)
(file: PartitionedFile) => {
val sharedConf = broadcastedHadoopConf.value.value
@@ -134,22 +136,42 @@ class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with
}
pushed.foreach(p => ParquetInputFormat.setFilterPredicate(sharedConf, p))
- val batchReader = new BatchReader(
- sharedConf,
- file,
- footer,
- capacity,
- requiredSchema,
- isCaseSensitive,
- useFieldId,
- ignoreMissingIds,
- datetimeRebaseSpec.mode == CORRECTED,
- partitionSchema,
- file.partitionValues,
- JavaConverters.mapAsJavaMap(metrics))
- val iter = new RecordReaderIterator(batchReader)
+ val recordBatchReader =
+ if (nativeRecordBatchReaderEnabled) {
+ val batchReader = new NativeBatchReader(
+ sharedConf,
+ file,
+ footer,
+ capacity,
+ requiredSchema,
+ isCaseSensitive,
+ useFieldId,
+ ignoreMissingIds,
+ datetimeRebaseSpec.mode == CORRECTED,
+ partitionSchema,
+ file.partitionValues,
+ JavaConverters.mapAsJavaMap(metrics))
+ batchReader.init()
+ batchReader
+ } else {
+ val batchReader = new BatchReader(
+ sharedConf,
+ file,
+ footer,
+ capacity,
+ requiredSchema,
+ isCaseSensitive,
+ useFieldId,
+ ignoreMissingIds,
+ datetimeRebaseSpec.mode == CORRECTED,
+ partitionSchema,
+ file.partitionValues,
+ JavaConverters.mapAsJavaMap(metrics))
+ batchReader.init()
+ batchReader
+ }
+ val iter = new RecordReaderIterator(recordBatchReader)
try {
- batchReader.init()
iter.asInstanceOf[Iterator[InternalRow]]
} catch {
case e: Throwable =>
diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
index b806b00809..244d24385e 100644
--- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
@@ -29,12 +29,14 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, Normalize
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition}
import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils
-import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometSinkPlaceHolder, CometSparkToColumnarExec, DecimalPrecision}
+import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometScanExec, CometSinkPlaceHolder, CometSparkToColumnarExec, DecimalPrecision}
import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec
import org.apache.spark.sql.execution
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec}
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDD, DataSourceRDDPartition}
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.window.WindowExec
@@ -59,7 +61,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
logWarning(s"Comet native execution is disabled due to: $reason")
}
- def supportedDataType(dt: DataType, allowStruct: Boolean = false): Boolean = dt match {
+ def supportedDataType(dt: DataType, allowStruct: Boolean = true): Boolean = dt match {
case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType |
_: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: DecimalType |
_: DateType | _: BooleanType | _: NullType =>
@@ -2540,6 +2542,83 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
childOp.foreach(result.addChildren)
op match {
+
+ // Fully native scan for V1
+ case scan: CometScanExec
+ if CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) == CometConf.SCAN_NATIVE_DATAFUSION =>
+ val nativeScanBuilder = OperatorOuterClass.NativeScan.newBuilder()
+ nativeScanBuilder.setSource(op.simpleStringWithNodeId())
+
+ val scanTypes = op.output.flatten { attr =>
+ serializeDataType(attr.dataType)
+ }
+
+ if (scanTypes.length == op.output.length) {
+ nativeScanBuilder.addAllFields(scanTypes.asJava)
+
+ // Sink operators don't have children
+ result.clearChildren()
+
+ // TODO remove flatMap and add error handling for unsupported data filters
+ val dataFilters = scan.dataFilters.flatMap(exprToProto(_, scan.output))
+ nativeScanBuilder.addAllDataFilters(dataFilters.asJava)
+
+ // TODO: modify CometNativeScan to generate the file partitions without instantiating RDD.
+ scan.inputRDD match {
+ case rdd: DataSourceRDD =>
+ val partitions = rdd.partitions
+ partitions.foreach(p => {
+ val inputPartitions = p.asInstanceOf[DataSourceRDDPartition].inputPartitions
+ inputPartitions.foreach(partition => {
+ partition2Proto(
+ partition.asInstanceOf[FilePartition],
+ nativeScanBuilder,
+ scan.relation.partitionSchema)
+ })
+ })
+ case rdd: FileScanRDD =>
+ rdd.filePartitions.foreach(partition => {
+ partition2Proto(partition, nativeScanBuilder, scan.relation.partitionSchema)
+ })
+ case _ =>
+ }
+
+ val partitionSchema = schema2Proto(scan.relation.partitionSchema.fields)
+ val requiredSchema = schema2Proto(scan.requiredSchema.fields)
+ val dataSchema = schema2Proto(scan.relation.dataSchema.fields)
+
+ val data_schema_idxs = scan.requiredSchema.fields.map(field => {
+ scan.relation.dataSchema.fieldIndex(field.name)
+ })
+ val partition_schema_idxs = Array
+ .range(
+ scan.relation.dataSchema.fields.length,
+ scan.relation.dataSchema.length + scan.relation.partitionSchema.fields.length)
+
+ val projection_vector = (data_schema_idxs ++ partition_schema_idxs).map(idx =>
+ idx.toLong.asInstanceOf[java.lang.Long])
+
+ nativeScanBuilder.addAllProjectionVector(projection_vector.toIterable.asJava)
+
+ // In `CometScanRule`, we ensure partitionSchema is supported.
+ assert(partitionSchema.length == scan.relation.partitionSchema.fields.length)
+
+ nativeScanBuilder.addAllDataSchema(dataSchema.toIterable.asJava)
+ nativeScanBuilder.addAllRequiredSchema(requiredSchema.toIterable.asJava)
+ nativeScanBuilder.addAllPartitionSchema(partitionSchema.toIterable.asJava)
+ nativeScanBuilder.setSessionTimezone(conf.getConfString("spark.sql.session.timeZone"))
+
+ Some(result.setNativeScan(nativeScanBuilder).build())
+
+ } else {
+ // There are unsupported scan type
+ val msg =
+ s"unsupported Comet operator: ${op.nodeName}, due to unsupported data types above"
+ emitWarning(msg)
+ withInfo(op, msg)
+ None
+ }
+
case ProjectExec(projectList, child) if CometConf.COMET_EXEC_PROJECT_ENABLED.get(conf) =>
val exprs = projectList.map(exprToProto(_, child.output))
@@ -3198,4 +3277,48 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
true
}
+
+ private def schema2Proto(
+ fields: Array[StructField]): Array[OperatorOuterClass.SparkStructField] = {
+ val fieldBuilder = OperatorOuterClass.SparkStructField.newBuilder()
+ fields.map(field => {
+ fieldBuilder.setName(field.name)
+ fieldBuilder.setDataType(serializeDataType(field.dataType).get)
+ fieldBuilder.setNullable(field.nullable)
+ fieldBuilder.build()
+ })
+ }
+
+ private def partition2Proto(
+ partition: FilePartition,
+ nativeScanBuilder: OperatorOuterClass.NativeScan.Builder,
+ partitionSchema: StructType): Unit = {
+ val partitionBuilder = OperatorOuterClass.SparkFilePartition.newBuilder()
+ partition.files.foreach(file => {
+ // Process the partition values
+ val partitionValues = file.partitionValues
+ assert(partitionValues.numFields == partitionSchema.length)
+ val partitionVals =
+ partitionValues.toSeq(partitionSchema).zipWithIndex.map { case (value, i) =>
+ val attr = partitionSchema(i)
+ val valueProto = exprToProto(Literal(value, attr.dataType), Seq.empty)
+ // In `CometScanRule`, we have already checked that all partition values are
+ // supported. So, we can safely use `get` here.
+ assert(
+ valueProto.isDefined,
+ s"Unsupported partition value: $value, type: ${attr.dataType}")
+ valueProto.get
+ }
+
+ val fileBuilder = OperatorOuterClass.SparkPartitionedFile.newBuilder()
+ partitionVals.foreach(fileBuilder.addPartitionValues)
+ fileBuilder
+ .setFilePath(file.filePath.toString)
+ .setStart(file.start)
+ .setLength(file.length)
+ .setFileSize(file.fileSize)
+ partitionBuilder.addPartitionedFile(fileBuilder.build())
+ })
+ nativeScanBuilder.addFilePartitions(partitionBuilder.build())
+ }
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala
new file mode 100644
index 0000000000..2fd7f12c24
--- /dev/null
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.{RDD, RDDOperationScope}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * A RDD that executes Spark SQL query in Comet native execution to generate ColumnarBatch.
+ */
+private[spark] class CometExecRDD(
+ sc: SparkContext,
+ partitionNum: Int,
+ var f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch])
+ extends RDD[ColumnarBatch](sc, Nil) {
+
+ override def compute(s: Partition, context: TaskContext): Iterator[ColumnarBatch] = {
+ f(Seq.empty, partitionNum, s.index)
+ }
+
+ override protected def getPartitions: Array[Partition] = {
+ Array.tabulate(partitionNum)(i =>
+ new Partition {
+ override def index: Int = i
+ })
+ }
+}
+
+object CometExecRDD {
+ def apply(sc: SparkContext, partitionNum: Int)(
+ f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch])
+ : RDD[ColumnarBatch] =
+ withScope(sc) {
+ new CometExecRDD(sc, partitionNum, f)
+ }
+
+ private[spark] def withScope[U](sc: SparkContext)(body: => U): U =
+ RDDOperationScope.withScope[U](sc)(body)
+}
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
new file mode 100644
index 0000000000..ed9e545dd1
--- /dev/null
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types._
+import org.apache.spark.util.collection._
+
+import com.google.common.base.Objects
+
+import org.apache.comet.DataTypeSupport
+import org.apache.comet.parquet.CometParquetFileFormat
+import org.apache.comet.serde.OperatorOuterClass.Operator
+
+/**
+ * Comet fully native scan node for DataSource V1.
+ */
+case class CometNativeScanExec(
+ override val nativeOp: Operator,
+ @transient relation: HadoopFsRelation,
+ override val output: Seq[Attribute],
+ requiredSchema: StructType,
+ partitionFilters: Seq[Expression],
+ optionalBucketSet: Option[BitSet],
+ optionalNumCoalescedBuckets: Option[Int],
+ dataFilters: Seq[Expression],
+ tableIdentifier: Option[TableIdentifier],
+ disableBucketedScan: Boolean = false,
+ originalPlan: FileSourceScanExec,
+ override val serializedPlanOpt: SerializedPlan)
+ extends CometLeafExec {
+
+ override def nodeName: String =
+ s"${super.nodeName}: ${tableIdentifier.map(_.toString).getOrElse("")}"
+
+ override def outputPartitioning: Partitioning =
+ UnknownPartitioning(originalPlan.inputRDD.getNumPartitions)
+ override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering
+
+ override def stringArgs: Iterator[Any] = Iterator(output)
+
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case other: CometNativeScanExec =>
+ this.output == other.output &&
+ this.serializedPlanOpt == other.serializedPlanOpt
+ case _ =>
+ false
+ }
+ }
+
+ override def hashCode(): Int = Objects.hashCode(output)
+}
+
+object CometNativeScanExec extends DataTypeSupport {
+ 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)
+ case other: AnyRef => other
+ case null => null
+ }
+ val newArgs = mapProductIterator(scanExec, transform(_))
+ val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec]
+ val batchScanExec = CometNativeScanExec(
+ nativeOp,
+ wrapped.relation,
+ wrapped.output,
+ wrapped.requiredSchema,
+ wrapped.partitionFilters,
+ wrapped.optionalBucketSet,
+ wrapped.optionalNumCoalescedBuckets,
+ wrapped.dataFilters,
+ wrapped.tableIdentifier,
+ wrapped.disableBucketedScan,
+ wrapped,
+ SerializedPlan(None))
+ scanExec.logicalLink.foreach(batchScanExec.setLogicalLink)
+ batchScanExec
+ }
+
+ override def isAdditionallySupported(dt: DataType): Boolean = {
+ // TODO add array and map
+ dt match {
+ case s: StructType => s.fields.map(_.dataType).forall(isTypeSupported)
+ case _ => false
+ }
+ }
+}
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala
index eb9d5cb314..497857d9d7 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala
@@ -131,8 +131,15 @@ case class CometScanExec(
// exposed for testing
lazy val bucketedScan: Boolean = wrapped.bucketedScan
- override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) =
- (wrapped.outputPartitioning, wrapped.outputOrdering)
+ override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = {
+ if (bucketedScan) {
+ (wrapped.outputPartitioning, wrapped.outputOrdering)
+ } else {
+ val files = selectedPartitions.flatMap(partition => partition.files)
+ val numPartitions = files.length
+ (UnknownPartitioning(numPartitions), wrapped.outputOrdering)
+ }
+ }
@transient
private lazy val pushedDownFilters = getPushedDownFilters(relation, dataFilters)
@@ -465,6 +472,19 @@ case class CometScanExec(
}
object CometScanExec extends DataTypeSupport {
+
+ override def isAdditionallySupported(dt: DataType): Boolean = {
+ if (CometConf.COMET_NATIVE_SCAN_IMPL.get() == CometConf.SCAN_NATIVE_ICEBERG_COMPAT) {
+ // TODO add array and map
+ dt match {
+ case s: StructType => s.fields.map(_.dataType).forall(isTypeSupported)
+ case _ => false
+ }
+ } else {
+ false
+ }
+ }
+
def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = {
// TreeNode.mapProductIterator is protected method.
def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = {
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 c70f7464ec..9e2ca987fa 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
@@ -242,53 +242,77 @@ abstract class CometNativeExec extends CometExec {
case _ => true
}
+ val containsBroadcastInput = sparkPlans.exists {
+ case _: CometBroadcastExchangeExec => true
+ case BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => true
+ case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => true
+ case _ => false
+ }
+
// If the first non broadcast plan is not found, it means all the plans are broadcast plans.
// This is not expected, so throw an exception.
- if (firstNonBroadcastPlan.isEmpty) {
+ if (containsBroadcastInput && firstNonBroadcastPlan.isEmpty) {
throw new CometRuntimeException(s"Cannot find the first non broadcast plan: $this")
}
// If the first non broadcast plan is found, we need to adjust the partition number of
// the broadcast plans to make sure they have the same partition number as the first non
// broadcast plan.
- val firstNonBroadcastPlanRDD = firstNonBroadcastPlan.get._1.executeColumnar()
- val firstNonBroadcastPlanNumPartitions = firstNonBroadcastPlanRDD.getNumPartitions
+ val firstNonBroadcastPlanNumPartitions =
+ firstNonBroadcastPlan.map(_._1.outputPartitioning.numPartitions)
// Spark doesn't need to zip Broadcast RDDs, so it doesn't schedule Broadcast RDDs with
// same partition number. But for Comet, we need to zip them so we need to adjust the
// partition number of Broadcast RDDs to make sure they have the same partition number.
- sparkPlans.zipWithIndex.foreach { case (plan, idx) =>
+ sparkPlans.zipWithIndex.foreach { case (plan, _) =>
plan match {
- case c: CometBroadcastExchangeExec =>
- inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar()
- case BroadcastQueryStageExec(_, c: CometBroadcastExchangeExec, _) =>
- inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar()
- case ReusedExchangeExec(_, c: CometBroadcastExchangeExec) =>
- inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar()
+ case c: CometBroadcastExchangeExec if firstNonBroadcastPlanNumPartitions.nonEmpty =>
+ inputs += c
+ .setNumPartitions(firstNonBroadcastPlanNumPartitions.get)
+ .executeColumnar()
+ case BroadcastQueryStageExec(_, c: CometBroadcastExchangeExec, _)
+ if firstNonBroadcastPlanNumPartitions.nonEmpty =>
+ inputs += c
+ .setNumPartitions(firstNonBroadcastPlanNumPartitions.get)
+ .executeColumnar()
+ case ReusedExchangeExec(_, c: CometBroadcastExchangeExec)
+ if firstNonBroadcastPlanNumPartitions.nonEmpty =>
+ inputs += c
+ .setNumPartitions(firstNonBroadcastPlanNumPartitions.get)
+ .executeColumnar()
case BroadcastQueryStageExec(
_,
ReusedExchangeExec(_, c: CometBroadcastExchangeExec),
- _) =>
- inputs += c.setNumPartitions(firstNonBroadcastPlanNumPartitions).executeColumnar()
- case _ if idx == firstNonBroadcastPlan.get._2 =>
- inputs += firstNonBroadcastPlanRDD
- case _ =>
+ _) if firstNonBroadcastPlanNumPartitions.nonEmpty =>
+ inputs += c
+ .setNumPartitions(firstNonBroadcastPlanNumPartitions.get)
+ .executeColumnar()
+ case _: CometNativeExec =>
+ // no-op
+ case _ if firstNonBroadcastPlanNumPartitions.nonEmpty =>
val rdd = plan.executeColumnar()
- if (rdd.getNumPartitions != firstNonBroadcastPlanNumPartitions) {
+ if (plan.outputPartitioning.numPartitions != firstNonBroadcastPlanNumPartitions.get) {
throw new CometRuntimeException(
s"Partition number mismatch: ${rdd.getNumPartitions} != " +
- s"$firstNonBroadcastPlanNumPartitions")
+ s"${firstNonBroadcastPlanNumPartitions.get}")
} else {
inputs += rdd
}
+ case _ =>
+ throw new CometRuntimeException(s"Unexpected plan: $plan")
}
}
- if (inputs.isEmpty) {
+ if (inputs.isEmpty && !sparkPlans.forall(_.isInstanceOf[CometNativeExec])) {
throw new CometRuntimeException(s"No input for CometNativeExec:\n $this")
}
- ZippedPartitionsRDD(sparkContext, inputs.toSeq)(createCometExecIter)
+ if (inputs.nonEmpty) {
+ ZippedPartitionsRDD(sparkContext, inputs.toSeq)(createCometExecIter)
+ } else {
+ val partitionNum = firstNonBroadcastPlanNumPartitions.get
+ CometExecRDD(sparkContext, partitionNum)(createCometExecIter)
+ }
}
}
@@ -312,10 +336,10 @@ abstract class CometNativeExec extends CometExec {
*/
def foreachUntilCometInput(plan: SparkPlan)(func: SparkPlan => Unit): Unit = {
plan match {
- case _: CometScanExec | _: CometBatchScanExec | _: ShuffleQueryStageExec |
- _: AQEShuffleReadExec | _: CometShuffleExchangeExec | _: CometUnionExec |
- _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | _: ReusedExchangeExec |
- _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec |
+ case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec |
+ _: ShuffleQueryStageExec | _: AQEShuffleReadExec | _: CometShuffleExchangeExec |
+ _: CometUnionExec | _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec |
+ _: ReusedExchangeExec | _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec |
_: CometSparkToColumnarExec =>
func(plan)
case _: CometPlan =>
@@ -395,6 +419,8 @@ abstract class CometNativeExec extends CometExec {
}
}
+abstract class CometLeafExec extends CometNativeExec with LeafExecNode
+
abstract class CometUnaryExec extends CometNativeExec with UnaryExecNode
abstract class CometBinaryExec extends CometNativeExec with BinaryExecNode
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt
new file mode 100644
index 0000000000..a1697ca9eb
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt
@@ -0,0 +1,212 @@
+== Physical Plan ==
+* ColumnarToRow (40)
++- CometTakeOrderedAndProject (39)
+ +- CometProject (38)
+ +- CometBroadcastHashJoin (37)
+ :- CometProject (33)
+ : +- CometBroadcastHashJoin (32)
+ : :- CometProject (27)
+ : : +- CometBroadcastHashJoin (26)
+ : : :- CometFilter (12)
+ : : : +- CometHashAggregate (11)
+ : : : +- CometExchange (10)
+ : : : +- CometHashAggregate (9)
+ : : : +- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_returns` (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (3)
+ : : +- CometBroadcastExchange (25)
+ : : +- CometFilter (24)
+ : : +- CometHashAggregate (23)
+ : : +- CometExchange (22)
+ : : +- CometHashAggregate (21)
+ : : +- CometHashAggregate (20)
+ : : +- CometExchange (19)
+ : : +- CometHashAggregate (18)
+ : : +- CometProject (17)
+ : : +- CometBroadcastHashJoin (16)
+ : : :- CometFilter (14)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`store_returns` (13)
+ : : +- ReusedExchange (15)
+ : +- CometBroadcastExchange (31)
+ : +- CometProject (30)
+ : +- CometFilter (29)
+ : +- CometNativeScan: `spark_catalog`.`default`.`store` (28)
+ +- CometBroadcastExchange (36)
+ +- CometFilter (35)
+ +- CometNativeScan: `spark_catalog`.`default`.`customer` (34)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`store_returns`
+Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+
+(2) CometFilter
+Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1))
+
+(3) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#5, d_year#6]
+Arguments: [d_date_sk#5, d_year#6]
+
+(4) CometFilter
+Input [2]: [d_date_sk#5, d_year#6]
+Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5))
+
+(5) CometProject
+Input [2]: [d_date_sk#5, d_year#6]
+Arguments: [d_date_sk#5], [d_date_sk#5]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#5]
+Arguments: [d_date_sk#5]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Right output [1]: [d_date_sk#5]
+Arguments: [sr_returned_date_sk#4], [d_date_sk#5], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#5]
+Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+
+(9) CometHashAggregate
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))]
+
+(10) CometExchange
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#7]
+Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(11) CometHashAggregate
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#7]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#3))]
+
+(12) CometFilter
+Input [3]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10]
+Condition : isnotnull(ctr_total_return#10)
+
+(13) CometNativeScan: `spark_catalog`.`default`.`store_returns`
+Output [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Arguments: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+
+(14) CometFilter
+Input [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Condition : isnotnull(sr_store_sk#12)
+
+(15) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#15]
+
+(16) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Right output [1]: [d_date_sk#15]
+Arguments: [sr_returned_date_sk#14], [d_date_sk#15], Inner, BuildRight
+
+(17) CometProject
+Input [5]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14, d_date_sk#15]
+Arguments: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13], [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13]
+
+(18) CometHashAggregate
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13]
+Keys [2]: [sr_customer_sk#11, sr_store_sk#12]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#13))]
+
+(19) CometExchange
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sum#16]
+Arguments: hashpartitioning(sr_customer_sk#11, sr_store_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(20) CometHashAggregate
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sum#16]
+Keys [2]: [sr_customer_sk#11, sr_store_sk#12]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#13))]
+
+(21) CometHashAggregate
+Input [2]: [ctr_store_sk#17, ctr_total_return#18]
+Keys [1]: [ctr_store_sk#17]
+Functions [1]: [partial_avg(ctr_total_return#18)]
+
+(22) CometExchange
+Input [3]: [ctr_store_sk#17, sum#19, count#20]
+Arguments: hashpartitioning(ctr_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(23) CometHashAggregate
+Input [3]: [ctr_store_sk#17, sum#19, count#20]
+Keys [1]: [ctr_store_sk#17]
+Functions [1]: [avg(ctr_total_return#18)]
+
+(24) CometFilter
+Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Condition : isnotnull((avg(ctr_total_return) * 1.2)#21)
+
+(25) CometBroadcastExchange
+Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+
+(26) CometBroadcastHashJoin
+Left output [3]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10]
+Right output [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [ctr_store_sk#9], [ctr_store_sk#17], Inner, (cast(ctr_total_return#10 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21), BuildRight
+
+(27) CometProject
+Input [5]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [ctr_customer_sk#8, ctr_store_sk#9], [ctr_customer_sk#8, ctr_store_sk#9]
+
+(28) CometNativeScan: `spark_catalog`.`default`.`store`
+Output [2]: [s_store_sk#22, s_state#23]
+Arguments: [s_store_sk#22, s_state#23]
+
+(29) CometFilter
+Input [2]: [s_store_sk#22, s_state#23]
+Condition : ((isnotnull(s_state#23) AND (s_state#23 = TN)) AND isnotnull(s_store_sk#22))
+
+(30) CometProject
+Input [2]: [s_store_sk#22, s_state#23]
+Arguments: [s_store_sk#22], [s_store_sk#22]
+
+(31) CometBroadcastExchange
+Input [1]: [s_store_sk#22]
+Arguments: [s_store_sk#22]
+
+(32) CometBroadcastHashJoin
+Left output [2]: [ctr_customer_sk#8, ctr_store_sk#9]
+Right output [1]: [s_store_sk#22]
+Arguments: [ctr_store_sk#9], [s_store_sk#22], Inner, BuildRight
+
+(33) CometProject
+Input [3]: [ctr_customer_sk#8, ctr_store_sk#9, s_store_sk#22]
+Arguments: [ctr_customer_sk#8], [ctr_customer_sk#8]
+
+(34) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [2]: [c_customer_sk#24, c_customer_id#25]
+Arguments: [c_customer_sk#24, c_customer_id#25]
+
+(35) CometFilter
+Input [2]: [c_customer_sk#24, c_customer_id#25]
+Condition : isnotnull(c_customer_sk#24)
+
+(36) CometBroadcastExchange
+Input [2]: [c_customer_sk#24, c_customer_id#25]
+Arguments: [c_customer_sk#24, c_customer_id#25]
+
+(37) CometBroadcastHashJoin
+Left output [1]: [ctr_customer_sk#8]
+Right output [2]: [c_customer_sk#24, c_customer_id#25]
+Arguments: [ctr_customer_sk#8], [c_customer_sk#24], Inner, BuildRight
+
+(38) CometProject
+Input [3]: [ctr_customer_sk#8, c_customer_sk#24, c_customer_id#25]
+Arguments: [c_customer_id#25], [c_customer_id#25]
+
+(39) CometTakeOrderedAndProject
+Input [1]: [c_customer_id#25]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#25 ASC NULLS FIRST], output=[c_customer_id#25]), [c_customer_id#25], 100, [c_customer_id#25 ASC NULLS FIRST], [c_customer_id#25]
+
+(40) ColumnarToRow [codegen id : 1]
+Input [1]: [c_customer_id#25]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..87657df444
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt
@@ -0,0 +1,42 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [c_customer_id]
+ CometProject [c_customer_id]
+ CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id]
+ CometProject [ctr_customer_sk]
+ CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk]
+ CometProject [ctr_customer_sk,ctr_store_sk]
+ CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk]
+ CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return]
+ CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))]
+ CometExchange [sr_customer_sk,sr_store_sk] #1
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_returns` [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometBroadcastExchange [d_date_sk] #2
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #3
+ CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk]
+ CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count,avg(ctr_total_return)]
+ CometExchange [ctr_store_sk] #4
+ CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return]
+ CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))]
+ CometExchange [sr_customer_sk,sr_store_sk] #5
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_returns` [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ ReusedExchange [d_date_sk] #2
+ CometBroadcastExchange [s_store_sk] #6
+ CometProject [s_store_sk]
+ CometFilter [s_store_sk,s_state]
+ CometNativeScan: `spark_catalog`.`default`.`store` [s_store_sk,s_state]
+ CometBroadcastExchange [c_customer_sk,c_customer_id] #7
+ CometFilter [c_customer_sk,c_customer_id]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_customer_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..ca868a928e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt
@@ -0,0 +1,229 @@
+== Physical Plan ==
+* ColumnarToRow (40)
++- CometTakeOrderedAndProject (39)
+ +- CometProject (38)
+ +- CometBroadcastHashJoin (37)
+ :- CometProject (33)
+ : +- CometBroadcastHashJoin (32)
+ : :- CometProject (27)
+ : : +- CometBroadcastHashJoin (26)
+ : : :- CometFilter (12)
+ : : : +- CometHashAggregate (11)
+ : : : +- CometExchange (10)
+ : : : +- CometHashAggregate (9)
+ : : : +- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_returns (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : +- CometBroadcastExchange (25)
+ : : +- CometFilter (24)
+ : : +- CometHashAggregate (23)
+ : : +- CometExchange (22)
+ : : +- CometHashAggregate (21)
+ : : +- CometHashAggregate (20)
+ : : +- CometExchange (19)
+ : : +- CometHashAggregate (18)
+ : : +- CometProject (17)
+ : : +- CometBroadcastHashJoin (16)
+ : : :- CometFilter (14)
+ : : : +- CometScan parquet spark_catalog.default.store_returns (13)
+ : : +- ReusedExchange (15)
+ : +- CometBroadcastExchange (31)
+ : +- CometProject (30)
+ : +- CometFilter (29)
+ : +- CometScan parquet spark_catalog.default.store (28)
+ +- CometBroadcastExchange (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.customer (34)
+
+
+(1) CometScan parquet spark_catalog.default.store_returns
+Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#4)]
+PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1))
+
+(3) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#5, d_year#6]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [d_date_sk#5, d_year#6]
+Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5))
+
+(5) CometProject
+Input [2]: [d_date_sk#5, d_year#6]
+Arguments: [d_date_sk#5], [d_date_sk#5]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#5]
+Arguments: [d_date_sk#5]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Right output [1]: [d_date_sk#5]
+Arguments: [sr_returned_date_sk#4], [d_date_sk#5], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#5]
+Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+
+(9) CometHashAggregate
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))]
+
+(10) CometExchange
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#7]
+Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(11) CometHashAggregate
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#7]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#3))]
+
+(12) CometFilter
+Input [3]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10]
+Condition : isnotnull(ctr_total_return#10)
+
+(13) CometScan parquet spark_catalog.default.store_returns
+Output [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#14)]
+PushedFilters: [IsNotNull(sr_store_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Condition : isnotnull(sr_store_sk#12)
+
+(15) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#15]
+
+(16) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14]
+Right output [1]: [d_date_sk#15]
+Arguments: [sr_returned_date_sk#14], [d_date_sk#15], Inner, BuildRight
+
+(17) CometProject
+Input [5]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13, sr_returned_date_sk#14, d_date_sk#15]
+Arguments: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13], [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13]
+
+(18) CometHashAggregate
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sr_return_amt#13]
+Keys [2]: [sr_customer_sk#11, sr_store_sk#12]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#13))]
+
+(19) CometExchange
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sum#16]
+Arguments: hashpartitioning(sr_customer_sk#11, sr_store_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(20) CometHashAggregate
+Input [3]: [sr_customer_sk#11, sr_store_sk#12, sum#16]
+Keys [2]: [sr_customer_sk#11, sr_store_sk#12]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#13))]
+
+(21) CometHashAggregate
+Input [2]: [ctr_store_sk#17, ctr_total_return#18]
+Keys [1]: [ctr_store_sk#17]
+Functions [1]: [partial_avg(ctr_total_return#18)]
+
+(22) CometExchange
+Input [3]: [ctr_store_sk#17, sum#19, count#20]
+Arguments: hashpartitioning(ctr_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(23) CometHashAggregate
+Input [3]: [ctr_store_sk#17, sum#19, count#20]
+Keys [1]: [ctr_store_sk#17]
+Functions [1]: [avg(ctr_total_return#18)]
+
+(24) CometFilter
+Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Condition : isnotnull((avg(ctr_total_return) * 1.2)#21)
+
+(25) CometBroadcastExchange
+Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+
+(26) CometBroadcastHashJoin
+Left output [3]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10]
+Right output [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [ctr_store_sk#9], [ctr_store_sk#17], Inner, (cast(ctr_total_return#10 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21), BuildRight
+
+(27) CometProject
+Input [5]: [ctr_customer_sk#8, ctr_store_sk#9, ctr_total_return#10, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#17]
+Arguments: [ctr_customer_sk#8, ctr_store_sk#9], [ctr_customer_sk#8, ctr_store_sk#9]
+
+(28) CometScan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#22, s_state#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [2]: [s_store_sk#22, s_state#23]
+Condition : ((isnotnull(s_state#23) AND (s_state#23 = TN)) AND isnotnull(s_store_sk#22))
+
+(30) CometProject
+Input [2]: [s_store_sk#22, s_state#23]
+Arguments: [s_store_sk#22], [s_store_sk#22]
+
+(31) CometBroadcastExchange
+Input [1]: [s_store_sk#22]
+Arguments: [s_store_sk#22]
+
+(32) CometBroadcastHashJoin
+Left output [2]: [ctr_customer_sk#8, ctr_store_sk#9]
+Right output [1]: [s_store_sk#22]
+Arguments: [ctr_store_sk#9], [s_store_sk#22], Inner, BuildRight
+
+(33) CometProject
+Input [3]: [ctr_customer_sk#8, ctr_store_sk#9, s_store_sk#22]
+Arguments: [ctr_customer_sk#8], [ctr_customer_sk#8]
+
+(34) CometScan parquet spark_catalog.default.customer
+Output [2]: [c_customer_sk#24, c_customer_id#25]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [2]: [c_customer_sk#24, c_customer_id#25]
+Condition : isnotnull(c_customer_sk#24)
+
+(36) CometBroadcastExchange
+Input [2]: [c_customer_sk#24, c_customer_id#25]
+Arguments: [c_customer_sk#24, c_customer_id#25]
+
+(37) CometBroadcastHashJoin
+Left output [1]: [ctr_customer_sk#8]
+Right output [2]: [c_customer_sk#24, c_customer_id#25]
+Arguments: [ctr_customer_sk#8], [c_customer_sk#24], Inner, BuildRight
+
+(38) CometProject
+Input [3]: [ctr_customer_sk#8, c_customer_sk#24, c_customer_id#25]
+Arguments: [c_customer_id#25], [c_customer_id#25]
+
+(39) CometTakeOrderedAndProject
+Input [1]: [c_customer_id#25]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#25 ASC NULLS FIRST], output=[c_customer_id#25]), [c_customer_id#25], 100, [c_customer_id#25 ASC NULLS FIRST], [c_customer_id#25]
+
+(40) ColumnarToRow [codegen id : 1]
+Input [1]: [c_customer_id#25]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..dd2d55266f
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt
@@ -0,0 +1,42 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [c_customer_id]
+ CometProject [c_customer_id]
+ CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id]
+ CometProject [ctr_customer_sk]
+ CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk]
+ CometProject [ctr_customer_sk,ctr_store_sk]
+ CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk]
+ CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return]
+ CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))]
+ CometExchange [sr_customer_sk,sr_store_sk] #1
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometBroadcastExchange [d_date_sk] #2
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #3
+ CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk]
+ CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count,avg(ctr_total_return)]
+ CometExchange [ctr_store_sk] #4
+ CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return]
+ CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))]
+ CometExchange [sr_customer_sk,sr_store_sk] #5
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ ReusedExchange [d_date_sk] #2
+ CometBroadcastExchange [s_store_sk] #6
+ CometProject [s_store_sk]
+ CometFilter [s_store_sk,s_state]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
+ CometBroadcastExchange [c_customer_sk,c_customer_id] #7
+ CometFilter [c_customer_sk,c_customer_id]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt
new file mode 100644
index 0000000000..7c92771145
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt
@@ -0,0 +1,214 @@
+== Physical Plan ==
+TakeOrderedAndProject (40)
++- * HashAggregate (39)
+ +- Exchange (38)
+ +- * HashAggregate (37)
+ +- * Project (36)
+ +- * BroadcastHashJoin Inner BuildRight (35)
+ :- * Project (30)
+ : +- * BroadcastHashJoin Inner BuildRight (29)
+ : :- * Project (23)
+ : : +- * Filter (22)
+ : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (21)
+ : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19)
+ : : : :- * ColumnarToRow (12)
+ : : : : +- CometBroadcastHashJoin (11)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometNativeScan: `spark_catalog`.`default`.`customer` (1)
+ : : : : +- CometBroadcastExchange (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometBroadcastHashJoin (8)
+ : : : : :- CometNativeScan: `spark_catalog`.`default`.`store_sales` (3)
+ : : : : +- CometBroadcastExchange (7)
+ : : : : +- CometProject (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (4)
+ : : : +- BroadcastExchange (18)
+ : : : +- * ColumnarToRow (17)
+ : : : +- CometProject (16)
+ : : : +- CometBroadcastHashJoin (15)
+ : : : :- CometNativeScan: `spark_catalog`.`default`.`web_sales` (13)
+ : : : +- ReusedExchange (14)
+ : : +- ReusedExchange (20)
+ : +- BroadcastExchange (28)
+ : +- * ColumnarToRow (27)
+ : +- CometProject (26)
+ : +- CometFilter (25)
+ : +- CometNativeScan: `spark_catalog`.`default`.`customer_address` (24)
+ +- BroadcastExchange (34)
+ +- * ColumnarToRow (33)
+ +- CometFilter (32)
+ +- CometNativeScan: `spark_catalog`.`default`.`customer_demographics` (31)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Arguments: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+
+(2) CometFilter
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4))
+
+(3) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Arguments: [ss_customer_sk#6, ss_sold_date_sk#7]
+
+(4) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Arguments: [d_date_sk#8, d_year#9, d_moy#10]
+
+(5) CometFilter
+Input [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 1)) AND (d_moy#10 <= 4)) AND isnotnull(d_date_sk#8))
+
+(6) CometProject
+Input [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Arguments: [d_date_sk#8], [d_date_sk#8]
+
+(7) CometBroadcastExchange
+Input [1]: [d_date_sk#8]
+Arguments: [d_date_sk#8]
+
+(8) CometBroadcastHashJoin
+Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Right output [1]: [d_date_sk#8]
+Arguments: [ss_sold_date_sk#7], [d_date_sk#8], Inner, BuildRight
+
+(9) CometProject
+Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#8]
+Arguments: [ss_customer_sk#6], [ss_customer_sk#6]
+
+(10) CometBroadcastExchange
+Input [1]: [ss_customer_sk#6]
+Arguments: [ss_customer_sk#6]
+
+(11) CometBroadcastHashJoin
+Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Right output [1]: [ss_customer_sk#6]
+Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight
+
+(12) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+
+(13) CometNativeScan: `spark_catalog`.`default`.`web_sales`
+Output [2]: [ws_bill_customer_sk#11, ws_sold_date_sk#12]
+Arguments: [ws_bill_customer_sk#11, ws_sold_date_sk#12]
+
+(14) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#13]
+
+(15) CometBroadcastHashJoin
+Left output [2]: [ws_bill_customer_sk#11, ws_sold_date_sk#12]
+Right output [1]: [d_date_sk#13]
+Arguments: [ws_sold_date_sk#12], [d_date_sk#13], Inner, BuildRight
+
+(16) CometProject
+Input [3]: [ws_bill_customer_sk#11, ws_sold_date_sk#12, d_date_sk#13]
+Arguments: [ws_bill_customer_sk#11], [ws_bill_customer_sk#11]
+
+(17) ColumnarToRow [codegen id : 1]
+Input [1]: [ws_bill_customer_sk#11]
+
+(18) BroadcastExchange
+Input [1]: [ws_bill_customer_sk#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1]
+
+(19) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [ws_bill_customer_sk#11]
+Join type: ExistenceJoin(exists#2)
+Join condition: None
+
+(20) ReusedExchange [Reuses operator id: 18]
+Output [1]: [cs_ship_customer_sk#14]
+
+(21) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [cs_ship_customer_sk#14]
+Join type: ExistenceJoin(exists#1)
+Join condition: None
+
+(22) Filter [codegen id : 5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+Condition : (exists#2 OR exists#1)
+
+(23) Project [codegen id : 5]
+Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+
+(24) CometNativeScan: `spark_catalog`.`default`.`customer_address`
+Output [2]: [ca_address_sk#15, ca_county#16]
+Arguments: [ca_address_sk#15, ca_county#16]
+
+(25) CometFilter
+Input [2]: [ca_address_sk#15, ca_county#16]
+Condition : (ca_county#16 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#15))
+
+(26) CometProject
+Input [2]: [ca_address_sk#15, ca_county#16]
+Arguments: [ca_address_sk#15], [ca_address_sk#15]
+
+(27) ColumnarToRow [codegen id : 3]
+Input [1]: [ca_address_sk#15]
+
+(28) BroadcastExchange
+Input [1]: [ca_address_sk#15]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(29) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_addr_sk#5]
+Right keys [1]: [ca_address_sk#15]
+Join type: Inner
+Join condition: None
+
+(30) Project [codegen id : 5]
+Output [1]: [c_current_cdemo_sk#4]
+Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#15]
+
+(31) CometNativeScan: `spark_catalog`.`default`.`customer_demographics`
+Output [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Arguments: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+
+(32) CometFilter
+Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Condition : isnotnull(cd_demo_sk#17)
+
+(33) ColumnarToRow [codegen id : 4]
+Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+
+(34) BroadcastExchange
+Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+
+(35) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_cdemo_sk#4]
+Right keys [1]: [cd_demo_sk#17]
+Join type: Inner
+Join condition: None
+
+(36) Project [codegen id : 5]
+Output [8]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+
+(37) HashAggregate [codegen id : 5]
+Input [8]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Keys [8]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Functions [1]: [partial_count(1)]
+Aggregate Attributes [1]: [count#26]
+Results [9]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#27]
+
+(38) Exchange
+Input [9]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#27]
+Arguments: hashpartitioning(cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(39) HashAggregate [codegen id : 6]
+Input [9]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#27]
+Keys [8]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#28]
+Results [14]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, count(1)#28 AS cnt1#29, cd_purchase_estimate#21, count(1)#28 AS cnt2#30, cd_credit_rating#22, count(1)#28 AS cnt3#31, cd_dep_count#23, count(1)#28 AS cnt4#32, cd_dep_employed_count#24, count(1)#28 AS cnt5#33, cd_dep_college_count#25, count(1)#28 AS cnt6#34]
+
+(40) TakeOrderedAndProject
+Input [14]: [cd_gender#18, cd_marital_status#19, cd_education_status#20, cnt1#29, cd_purchase_estimate#21, cnt2#30, cd_credit_rating#22, cnt3#31, cd_dep_count#23, cnt4#32, cd_dep_employed_count#24, cnt5#33, cd_dep_college_count#25, cnt6#34]
+Arguments: 100, [cd_gender#18 ASC NULLS FIRST, cd_marital_status#19 ASC NULLS FIRST, cd_education_status#20 ASC NULLS FIRST, cd_purchase_estimate#21 ASC NULLS FIRST, cd_credit_rating#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [cd_gender#18, cd_marital_status#19, cd_education_status#20, cnt1#29, cd_purchase_estimate#21, cnt2#30, cd_credit_rating#22, cnt3#31, cd_dep_count#23, cnt4#32, cd_dep_employed_count#24, cnt5#33, cd_dep_college_count#25, cnt6#34]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..228afd2bac
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt
@@ -0,0 +1,54 @@
+TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6]
+ WholeStageCodegen (6)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count]
+ InputAdapter
+ Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1
+ WholeStageCodegen (5)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count]
+ Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ Project [c_current_cdemo_sk]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [c_current_cdemo_sk,c_current_addr_sk]
+ Filter [exists,exists]
+ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk]
+ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ ColumnarToRow
+ InputAdapter
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometBroadcastExchange [ss_customer_sk] #2
+ CometProject [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_customer_sk,ss_sold_date_sk]
+ CometBroadcastExchange [d_date_sk] #3
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`web_sales` [ws_bill_customer_sk,ws_sold_date_sk]
+ ReusedExchange [d_date_sk] #3
+ InputAdapter
+ ReusedExchange [cs_ship_customer_sk] #4
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
+ CometNativeScan: `spark_catalog`.`default`.`customer_address` [ca_address_sk,ca_county]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ CometNativeScan: `spark_catalog`.`default`.`customer_demographics` [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..07599dfe5e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt
@@ -0,0 +1,260 @@
+== Physical Plan ==
+TakeOrderedAndProject (45)
++- * HashAggregate (44)
+ +- Exchange (43)
+ +- * HashAggregate (42)
+ +- * Project (41)
+ +- * BroadcastHashJoin Inner BuildRight (40)
+ :- * Project (35)
+ : +- * BroadcastHashJoin Inner BuildRight (34)
+ : :- * Project (28)
+ : : +- * Filter (27)
+ : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26)
+ : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19)
+ : : : :- * ColumnarToRow (12)
+ : : : : +- CometBroadcastHashJoin (11)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.customer (1)
+ : : : : +- CometBroadcastExchange (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometBroadcastHashJoin (8)
+ : : : : :- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : : +- CometBroadcastExchange (7)
+ : : : : +- CometProject (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (4)
+ : : : +- BroadcastExchange (18)
+ : : : +- * ColumnarToRow (17)
+ : : : +- CometProject (16)
+ : : : +- CometBroadcastHashJoin (15)
+ : : : :- CometScan parquet spark_catalog.default.web_sales (13)
+ : : : +- ReusedExchange (14)
+ : : +- BroadcastExchange (25)
+ : : +- * ColumnarToRow (24)
+ : : +- CometProject (23)
+ : : +- CometBroadcastHashJoin (22)
+ : : :- CometScan parquet spark_catalog.default.catalog_sales (20)
+ : : +- ReusedExchange (21)
+ : +- BroadcastExchange (33)
+ : +- * ColumnarToRow (32)
+ : +- CometProject (31)
+ : +- CometFilter (30)
+ : +- CometScan parquet spark_catalog.default.customer_address (29)
+ +- BroadcastExchange (39)
+ +- * ColumnarToRow (38)
+ +- CometFilter (37)
+ +- CometScan parquet spark_catalog.default.customer_demographics (36)
+
+
+(1) CometScan parquet spark_catalog.default.customer
+Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4))
+
+(3) CometScan parquet spark_catalog.default.store_sales
+Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#7)]
+ReadSchema: struct
+
+(4) CometScan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 1)) AND (d_moy#10 <= 4)) AND isnotnull(d_date_sk#8))
+
+(6) CometProject
+Input [3]: [d_date_sk#8, d_year#9, d_moy#10]
+Arguments: [d_date_sk#8], [d_date_sk#8]
+
+(7) CometBroadcastExchange
+Input [1]: [d_date_sk#8]
+Arguments: [d_date_sk#8]
+
+(8) CometBroadcastHashJoin
+Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Right output [1]: [d_date_sk#8]
+Arguments: [ss_sold_date_sk#7], [d_date_sk#8], Inner, BuildRight
+
+(9) CometProject
+Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#8]
+Arguments: [ss_customer_sk#6], [ss_customer_sk#6]
+
+(10) CometBroadcastExchange
+Input [1]: [ss_customer_sk#6]
+Arguments: [ss_customer_sk#6]
+
+(11) CometBroadcastHashJoin
+Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Right output [1]: [ss_customer_sk#6]
+Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight
+
+(12) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+
+(13) CometScan parquet spark_catalog.default.web_sales
+Output [2]: [ws_bill_customer_sk#11, ws_sold_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#12)]
+ReadSchema: struct
+
+(14) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#13]
+
+(15) CometBroadcastHashJoin
+Left output [2]: [ws_bill_customer_sk#11, ws_sold_date_sk#12]
+Right output [1]: [d_date_sk#13]
+Arguments: [ws_sold_date_sk#12], [d_date_sk#13], Inner, BuildRight
+
+(16) CometProject
+Input [3]: [ws_bill_customer_sk#11, ws_sold_date_sk#12, d_date_sk#13]
+Arguments: [ws_bill_customer_sk#11], [ws_bill_customer_sk#11]
+
+(17) ColumnarToRow [codegen id : 1]
+Input [1]: [ws_bill_customer_sk#11]
+
+(18) BroadcastExchange
+Input [1]: [ws_bill_customer_sk#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1]
+
+(19) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [ws_bill_customer_sk#11]
+Join type: ExistenceJoin(exists#2)
+Join condition: None
+
+(20) CometScan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#15)]
+ReadSchema: struct
+
+(21) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#16]
+
+(22) CometBroadcastHashJoin
+Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15]
+Right output [1]: [d_date_sk#16]
+Arguments: [cs_sold_date_sk#15], [d_date_sk#16], Inner, BuildRight
+
+(23) CometProject
+Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#16]
+Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14]
+
+(24) ColumnarToRow [codegen id : 2]
+Input [1]: [cs_ship_customer_sk#14]
+
+(25) BroadcastExchange
+Input [1]: [cs_ship_customer_sk#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(26) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [cs_ship_customer_sk#14]
+Join type: ExistenceJoin(exists#1)
+Join condition: None
+
+(27) Filter [codegen id : 5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+Condition : (exists#2 OR exists#1)
+
+(28) Project [codegen id : 5]
+Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+
+(29) CometScan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#17, ca_county#18]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(30) CometFilter
+Input [2]: [ca_address_sk#17, ca_county#18]
+Condition : (ca_county#18 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#17))
+
+(31) CometProject
+Input [2]: [ca_address_sk#17, ca_county#18]
+Arguments: [ca_address_sk#17], [ca_address_sk#17]
+
+(32) ColumnarToRow [codegen id : 3]
+Input [1]: [ca_address_sk#17]
+
+(33) BroadcastExchange
+Input [1]: [ca_address_sk#17]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+(34) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_addr_sk#5]
+Right keys [1]: [ca_address_sk#17]
+Join type: Inner
+Join condition: None
+
+(35) Project [codegen id : 5]
+Output [1]: [c_current_cdemo_sk#4]
+Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#17]
+
+(36) CometScan parquet spark_catalog.default.customer_demographics
+Output [9]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(37) CometFilter
+Input [9]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Condition : isnotnull(cd_demo_sk#19)
+
+(38) ColumnarToRow [codegen id : 4]
+Input [9]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+
+(39) BroadcastExchange
+Input [9]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+
+(40) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_cdemo_sk#4]
+Right keys [1]: [cd_demo_sk#19]
+Join type: Inner
+Join condition: None
+
+(41) Project [codegen id : 5]
+Output [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+
+(42) HashAggregate [codegen id : 5]
+Input [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Keys [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Functions [1]: [partial_count(1)]
+Aggregate Attributes [1]: [count#28]
+Results [9]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#29]
+
+(43) Exchange
+Input [9]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#29]
+Arguments: hashpartitioning(cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(44) HashAggregate [codegen id : 6]
+Input [9]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#29]
+Keys [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#23, cd_credit_rating#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#30]
+Results [14]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, count(1)#30 AS cnt1#31, cd_purchase_estimate#23, count(1)#30 AS cnt2#32, cd_credit_rating#24, count(1)#30 AS cnt3#33, cd_dep_count#25, count(1)#30 AS cnt4#34, cd_dep_employed_count#26, count(1)#30 AS cnt5#35, cd_dep_college_count#27, count(1)#30 AS cnt6#36]
+
+(45) TakeOrderedAndProject
+Input [14]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cnt1#31, cd_purchase_estimate#23, cnt2#32, cd_credit_rating#24, cnt3#33, cd_dep_count#25, cnt4#34, cd_dep_employed_count#26, cnt5#35, cd_dep_college_count#27, cnt6#36]
+Arguments: 100, [cd_gender#20 ASC NULLS FIRST, cd_marital_status#21 ASC NULLS FIRST, cd_education_status#22 ASC NULLS FIRST, cd_purchase_estimate#23 ASC NULLS FIRST, cd_credit_rating#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [cd_gender#20, cd_marital_status#21, cd_education_status#22, cnt1#31, cd_purchase_estimate#23, cnt2#32, cd_credit_rating#24, cnt3#33, cd_dep_count#25, cnt4#34, cd_dep_employed_count#26, cnt5#35, cd_dep_college_count#27, cnt6#36]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..f388a0a786
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt
@@ -0,0 +1,61 @@
+TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6]
+ WholeStageCodegen (6)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count]
+ InputAdapter
+ Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1
+ WholeStageCodegen (5)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count]
+ Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ Project [c_current_cdemo_sk]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [c_current_cdemo_sk,c_current_addr_sk]
+ Filter [exists,exists]
+ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk]
+ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ ColumnarToRow
+ InputAdapter
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometBroadcastExchange [ss_customer_sk] #2
+ CometProject [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
+ CometBroadcastExchange [d_date_sk] #3
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk]
+ ReusedExchange [d_date_sk] #3
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_ship_customer_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk]
+ ReusedExchange [d_date_sk] #3
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt
new file mode 100644
index 0000000000..d4ba47247d
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt
@@ -0,0 +1,331 @@
+== Physical Plan ==
+* ColumnarToRow (63)
++- CometTakeOrderedAndProject (62)
+ +- CometProject (61)
+ +- CometBroadcastHashJoin (60)
+ :- CometProject (47)
+ : +- CometBroadcastHashJoin (46)
+ : :- CometProject (32)
+ : : +- CometBroadcastHashJoin (31)
+ : : :- CometFilter (16)
+ : : : +- CometHashAggregate (15)
+ : : : +- CometExchange (14)
+ : : : +- CometHashAggregate (13)
+ : : : +- CometProject (12)
+ : : : +- CometBroadcastHashJoin (11)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometNativeScan: `spark_catalog`.`default`.`customer` (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (3)
+ : : : +- CometBroadcastExchange (10)
+ : : : +- CometFilter (9)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (8)
+ : : +- CometBroadcastExchange (30)
+ : : +- CometHashAggregate (29)
+ : : +- CometExchange (28)
+ : : +- CometHashAggregate (27)
+ : : +- CometProject (26)
+ : : +- CometBroadcastHashJoin (25)
+ : : :- CometProject (21)
+ : : : +- CometBroadcastHashJoin (20)
+ : : : :- CometFilter (18)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`customer` (17)
+ : : : +- ReusedExchange (19)
+ : : +- CometBroadcastExchange (24)
+ : : +- CometFilter (23)
+ : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (22)
+ : +- CometBroadcastExchange (45)
+ : +- CometFilter (44)
+ : +- CometHashAggregate (43)
+ : +- CometExchange (42)
+ : +- CometHashAggregate (41)
+ : +- CometProject (40)
+ : +- CometBroadcastHashJoin (39)
+ : :- CometProject (37)
+ : : +- CometBroadcastHashJoin (36)
+ : : :- CometFilter (34)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`customer` (33)
+ : : +- ReusedExchange (35)
+ : +- ReusedExchange (38)
+ +- CometBroadcastExchange (59)
+ +- CometHashAggregate (58)
+ +- CometExchange (57)
+ +- CometHashAggregate (56)
+ +- CometProject (55)
+ +- CometBroadcastHashJoin (54)
+ :- CometProject (52)
+ : +- CometBroadcastHashJoin (51)
+ : :- CometFilter (49)
+ : : +- CometNativeScan: `spark_catalog`.`default`.`customer` (48)
+ : +- ReusedExchange (50)
+ +- ReusedExchange (53)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Arguments: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+
+(2) CometFilter
+Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))
+
+(3) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(4) CometFilter
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Condition : isnotnull(ss_customer_sk#9)
+
+(5) CometBroadcastExchange
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight
+
+(7) CometProject
+Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(8) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#13, d_year#14]
+Arguments: [d_date_sk#13, d_year#14]
+
+(9) CometFilter
+Input [2]: [d_date_sk#13, d_year#14]
+Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13))
+
+(10) CometBroadcastExchange
+Input [2]: [d_date_sk#13, d_year#14]
+Arguments: [d_date_sk#13, d_year#14]
+
+(11) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Right output [2]: [d_date_sk#13, d_year#14]
+Arguments: [ss_sold_date_sk#12], [d_date_sk#13], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#13, d_year#14]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14]
+
+(13) CometHashAggregate
+Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+
+(14) CometExchange
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#15]
+Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(15) CometHashAggregate
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#15]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+
+(16) CometFilter
+Input [2]: [customer_id#16, year_total#17]
+Condition : (isnotnull(year_total#17) AND (year_total#17 > 0.00))
+
+(17) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Arguments: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+
+(18) CometFilter
+Input [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_customer_id#19))
+
+(19) ReusedExchange [Reuses operator id: 5]
+Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+
+(20) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Right output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Arguments: [c_customer_sk#18], [ss_customer_sk#26], Inner, BuildRight
+
+(21) CometProject
+Input [12]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Arguments: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29], [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+
+(22) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#30, d_year#31]
+Arguments: [d_date_sk#30, d_year#31]
+
+(23) CometFilter
+Input [2]: [d_date_sk#30, d_year#31]
+Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_sk#30))
+
+(24) CometBroadcastExchange
+Input [2]: [d_date_sk#30, d_year#31]
+Arguments: [d_date_sk#30, d_year#31]
+
+(25) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Right output [2]: [d_date_sk#30, d_year#31]
+Arguments: [ss_sold_date_sk#29], [d_date_sk#30], Inner, BuildRight
+
+(26) CometProject
+Input [12]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#30, d_year#31]
+Arguments: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31], [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31]
+
+(27) CometHashAggregate
+Input [10]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31]
+Keys [8]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#28 - ss_ext_discount_amt#27)))]
+
+(28) CometExchange
+Input [9]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, sum#32]
+Arguments: hashpartitioning(c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(29) CometHashAggregate
+Input [9]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, sum#32]
+Keys [8]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#28 - ss_ext_discount_amt#27)))]
+
+(30) CometBroadcastExchange
+Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+
+(31) CometBroadcastHashJoin
+Left output [2]: [customer_id#16, year_total#17]
+Right output [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#16], [customer_id#33], Inner, BuildRight
+
+(32) CometProject
+Input [5]: [customer_id#16, year_total#17, customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35], [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35]
+
+(33) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Arguments: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+
+(34) CometFilter
+Input [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Condition : (isnotnull(c_customer_sk#36) AND isnotnull(c_customer_id#37))
+
+(35) ReusedExchange [Reuses operator id: 5]
+Output [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+
+(36) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Right output [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Arguments: [c_customer_sk#36], [ws_bill_customer_sk#44], Inner, BuildRight
+
+(37) CometProject
+Input [12]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Arguments: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47], [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+
+(38) ReusedExchange [Reuses operator id: 10]
+Output [2]: [d_date_sk#48, d_year#49]
+
+(39) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Right output [2]: [d_date_sk#48, d_year#49]
+Arguments: [ws_sold_date_sk#47], [d_date_sk#48], Inner, BuildRight
+
+(40) CometProject
+Input [12]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49]
+Arguments: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49], [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49]
+
+(41) CometHashAggregate
+Input [10]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49]
+Keys [8]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#46 - ws_ext_discount_amt#45)))]
+
+(42) CometExchange
+Input [9]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, sum#50]
+Arguments: hashpartitioning(c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(43) CometHashAggregate
+Input [9]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, sum#50]
+Keys [8]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#46 - ws_ext_discount_amt#45)))]
+
+(44) CometFilter
+Input [2]: [customer_id#51, year_total#52]
+Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00))
+
+(45) CometBroadcastExchange
+Input [2]: [customer_id#51, year_total#52]
+Arguments: [customer_id#51, year_total#52]
+
+(46) CometBroadcastHashJoin
+Left output [4]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35]
+Right output [2]: [customer_id#51, year_total#52]
+Arguments: [customer_id#16], [customer_id#51], Inner, BuildRight
+
+(47) CometProject
+Input [6]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, customer_id#51, year_total#52]
+Arguments: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52], [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52]
+
+(48) CometNativeScan: `spark_catalog`.`default`.`customer`
+Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Arguments: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+
+(49) CometFilter
+Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Condition : (isnotnull(c_customer_sk#53) AND isnotnull(c_customer_id#54))
+
+(50) ReusedExchange [Reuses operator id: 5]
+Output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+
+(51) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Right output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Arguments: [c_customer_sk#53], [ws_bill_customer_sk#61], Inner, BuildRight
+
+(52) CometProject
+Input [12]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+
+(53) ReusedExchange [Reuses operator id: 24]
+Output [2]: [d_date_sk#65, d_year#66]
+
+(54) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Right output [2]: [d_date_sk#65, d_year#66]
+Arguments: [ws_sold_date_sk#64], [d_date_sk#65], Inner, BuildRight
+
+(55) CometProject
+Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66]
+Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66]
+
+(56) CometHashAggregate
+Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66]
+Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))]
+
+(57) CometExchange
+Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, sum#67]
+Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(58) CometHashAggregate
+Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, sum#67]
+Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))]
+
+(59) CometBroadcastExchange
+Input [2]: [customer_id#68, year_total#69]
+Arguments: [customer_id#68, year_total#69]
+
+(60) CometBroadcastHashJoin
+Left output [5]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52]
+Right output [2]: [customer_id#68, year_total#69]
+Arguments: [customer_id#16], [customer_id#68], Inner, (CASE WHEN (year_total#52 > 0.00) THEN (year_total#69 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END), BuildRight
+
+(61) CometProject
+Input [7]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52, customer_id#68, year_total#69]
+Arguments: [customer_preferred_cust_flag#34], [customer_preferred_cust_flag#34]
+
+(62) CometTakeOrderedAndProject
+Input [1]: [customer_preferred_cust_flag#34]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#34 ASC NULLS FIRST], output=[customer_preferred_cust_flag#34]), [customer_preferred_cust_flag#34], 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34]
+
+(63) ColumnarToRow [codegen id : 1]
+Input [1]: [customer_preferred_cust_flag#34]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..7bfc93c3e9
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt
@@ -0,0 +1,65 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [customer_preferred_cust_flag]
+ CometProject [customer_preferred_cust_flag]
+ CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total]
+ CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total]
+ CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total]
+ CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total]
+ CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total]
+ CometFilter [customer_id,year_total]
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #3
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #4
+ CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ ReusedExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometBroadcastExchange [d_date_sk,d_year] #6
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ CometBroadcastExchange [customer_id,year_total] #7
+ CometFilter [customer_id,year_total]
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #2
+ ReusedExchange [d_date_sk,d_year] #3
+ CometBroadcastExchange [customer_id,year_total] #9
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #10
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometNativeScan: `spark_catalog`.`default`.`customer` [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #2
+ ReusedExchange [d_date_sk,d_year] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..7e598f18a0
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt
@@ -0,0 +1,368 @@
+== Physical Plan ==
+* ColumnarToRow (65)
++- CometTakeOrderedAndProject (64)
+ +- CometProject (63)
+ +- CometBroadcastHashJoin (62)
+ :- CometProject (49)
+ : +- CometBroadcastHashJoin (48)
+ : :- CometProject (32)
+ : : +- CometBroadcastHashJoin (31)
+ : : :- CometFilter (16)
+ : : : +- CometHashAggregate (15)
+ : : : +- CometExchange (14)
+ : : : +- CometHashAggregate (13)
+ : : : +- CometProject (12)
+ : : : +- CometBroadcastHashJoin (11)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.customer (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : +- CometBroadcastExchange (10)
+ : : : +- CometFilter (9)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (8)
+ : : +- CometBroadcastExchange (30)
+ : : +- CometHashAggregate (29)
+ : : +- CometExchange (28)
+ : : +- CometHashAggregate (27)
+ : : +- CometProject (26)
+ : : +- CometBroadcastHashJoin (25)
+ : : :- CometProject (21)
+ : : : +- CometBroadcastHashJoin (20)
+ : : : :- CometFilter (18)
+ : : : : +- CometScan parquet spark_catalog.default.customer (17)
+ : : : +- ReusedExchange (19)
+ : : +- CometBroadcastExchange (24)
+ : : +- CometFilter (23)
+ : : +- CometScan parquet spark_catalog.default.date_dim (22)
+ : +- CometBroadcastExchange (47)
+ : +- CometFilter (46)
+ : +- CometHashAggregate (45)
+ : +- CometExchange (44)
+ : +- CometHashAggregate (43)
+ : +- CometProject (42)
+ : +- CometBroadcastHashJoin (41)
+ : :- CometProject (39)
+ : : +- CometBroadcastHashJoin (38)
+ : : :- CometFilter (34)
+ : : : +- CometScan parquet spark_catalog.default.customer (33)
+ : : +- CometBroadcastExchange (37)
+ : : +- CometFilter (36)
+ : : +- CometScan parquet spark_catalog.default.web_sales (35)
+ : +- ReusedExchange (40)
+ +- CometBroadcastExchange (61)
+ +- CometHashAggregate (60)
+ +- CometExchange (59)
+ +- CometHashAggregate (58)
+ +- CometProject (57)
+ +- CometBroadcastHashJoin (56)
+ :- CometProject (54)
+ : +- CometBroadcastHashJoin (53)
+ : :- CometFilter (51)
+ : : +- CometScan parquet spark_catalog.default.customer (50)
+ : +- ReusedExchange (52)
+ +- ReusedExchange (55)
+
+
+(1) CometScan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))
+
+(3) CometScan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#12)]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Condition : isnotnull(ss_customer_sk#9)
+
+(5) CometBroadcastExchange
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight
+
+(7) CometProject
+Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(8) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#13, d_year#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [d_date_sk#13, d_year#14]
+Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13))
+
+(10) CometBroadcastExchange
+Input [2]: [d_date_sk#13, d_year#14]
+Arguments: [d_date_sk#13, d_year#14]
+
+(11) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Right output [2]: [d_date_sk#13, d_year#14]
+Arguments: [ss_sold_date_sk#12], [d_date_sk#13], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#13, d_year#14]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14]
+
+(13) CometHashAggregate
+Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#14]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+
+(14) CometExchange
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#15]
+Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(15) CometHashAggregate
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#15]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#14, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+
+(16) CometFilter
+Input [2]: [customer_id#16, year_total#17]
+Condition : (isnotnull(year_total#17) AND (year_total#17 > 0.00))
+
+(17) CometScan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(18) CometFilter
+Input [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_customer_id#19))
+
+(19) ReusedExchange [Reuses operator id: 5]
+Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+
+(20) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Right output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Arguments: [c_customer_sk#18], [ss_customer_sk#26], Inner, BuildRight
+
+(21) CometProject
+Input [12]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Arguments: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29], [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+
+(22) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#30, d_year#31]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(23) CometFilter
+Input [2]: [d_date_sk#30, d_year#31]
+Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_sk#30))
+
+(24) CometBroadcastExchange
+Input [2]: [d_date_sk#30, d_year#31]
+Arguments: [d_date_sk#30, d_year#31]
+
+(25) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29]
+Right output [2]: [d_date_sk#30, d_year#31]
+Arguments: [ss_sold_date_sk#29], [d_date_sk#30], Inner, BuildRight
+
+(26) CometProject
+Input [12]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#30, d_year#31]
+Arguments: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31], [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31]
+
+(27) CometHashAggregate
+Input [10]: [c_customer_id#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#31]
+Keys [8]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#28 - ss_ext_discount_amt#27)))]
+
+(28) CometExchange
+Input [9]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, sum#32]
+Arguments: hashpartitioning(c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(29) CometHashAggregate
+Input [9]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25, sum#32]
+Keys [8]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#31, c_preferred_cust_flag#22, c_birth_country#23, c_login#24, c_email_address#25]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#28 - ss_ext_discount_amt#27)))]
+
+(30) CometBroadcastExchange
+Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+
+(31) CometBroadcastHashJoin
+Left output [2]: [customer_id#16, year_total#17]
+Right output [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#16], [customer_id#33], Inner, BuildRight
+
+(32) CometProject
+Input [5]: [customer_id#16, year_total#17, customer_id#33, customer_preferred_cust_flag#34, year_total#35]
+Arguments: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35], [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35]
+
+(33) CometScan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Condition : (isnotnull(c_customer_sk#36) AND isnotnull(c_customer_id#37))
+
+(35) CometScan parquet spark_catalog.default.web_sales
+Output [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#47)]
+PushedFilters: [IsNotNull(ws_bill_customer_sk)]
+ReadSchema: struct
+
+(36) CometFilter
+Input [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Condition : isnotnull(ws_bill_customer_sk#44)
+
+(37) CometBroadcastExchange
+Input [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Arguments: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+
+(38) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43]
+Right output [4]: [ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Arguments: [c_customer_sk#36], [ws_bill_customer_sk#44], Inner, BuildRight
+
+(39) CometProject
+Input [12]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_bill_customer_sk#44, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Arguments: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47], [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+
+(40) ReusedExchange [Reuses operator id: 10]
+Output [2]: [d_date_sk#48, d_year#49]
+
+(41) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47]
+Right output [2]: [d_date_sk#48, d_year#49]
+Arguments: [ws_sold_date_sk#47], [d_date_sk#48], Inner, BuildRight
+
+(42) CometProject
+Input [12]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49]
+Arguments: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49], [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49]
+
+(43) CometHashAggregate
+Input [10]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, ws_ext_discount_amt#45, ws_ext_list_price#46, d_year#49]
+Keys [8]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#46 - ws_ext_discount_amt#45)))]
+
+(44) CometExchange
+Input [9]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, sum#50]
+Arguments: hashpartitioning(c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(45) CometHashAggregate
+Input [9]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49, sum#50]
+Keys [8]: [c_customer_id#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_country#41, c_login#42, c_email_address#43, d_year#49]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#46 - ws_ext_discount_amt#45)))]
+
+(46) CometFilter
+Input [2]: [customer_id#51, year_total#52]
+Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00))
+
+(47) CometBroadcastExchange
+Input [2]: [customer_id#51, year_total#52]
+Arguments: [customer_id#51, year_total#52]
+
+(48) CometBroadcastHashJoin
+Left output [4]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35]
+Right output [2]: [customer_id#51, year_total#52]
+Arguments: [customer_id#16], [customer_id#51], Inner, BuildRight
+
+(49) CometProject
+Input [6]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, customer_id#51, year_total#52]
+Arguments: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52], [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52]
+
+(50) CometScan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(51) CometFilter
+Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Condition : (isnotnull(c_customer_sk#53) AND isnotnull(c_customer_id#54))
+
+(52) ReusedExchange [Reuses operator id: 37]
+Output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+
+(53) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60]
+Right output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Arguments: [c_customer_sk#53], [ws_bill_customer_sk#61], Inner, BuildRight
+
+(54) CometProject
+Input [12]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+
+(55) ReusedExchange [Reuses operator id: 24]
+Output [2]: [d_date_sk#65, d_year#66]
+
+(56) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64]
+Right output [2]: [d_date_sk#65, d_year#66]
+Arguments: [ws_sold_date_sk#64], [d_date_sk#65], Inner, BuildRight
+
+(57) CometProject
+Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66]
+Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66]
+
+(58) CometHashAggregate
+Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66]
+Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))]
+
+(59) CometExchange
+Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, sum#67]
+Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(60) CometHashAggregate
+Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66, sum#67]
+Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#66]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))]
+
+(61) CometBroadcastExchange
+Input [2]: [customer_id#68, year_total#69]
+Arguments: [customer_id#68, year_total#69]
+
+(62) CometBroadcastHashJoin
+Left output [5]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52]
+Right output [2]: [customer_id#68, year_total#69]
+Arguments: [customer_id#16], [customer_id#68], Inner, (CASE WHEN (year_total#52 > 0.00) THEN (year_total#69 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END), BuildRight
+
+(63) CometProject
+Input [7]: [customer_id#16, year_total#17, customer_preferred_cust_flag#34, year_total#35, year_total#52, customer_id#68, year_total#69]
+Arguments: [customer_preferred_cust_flag#34], [customer_preferred_cust_flag#34]
+
+(64) CometTakeOrderedAndProject
+Input [1]: [customer_preferred_cust_flag#34]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#34 ASC NULLS FIRST], output=[customer_preferred_cust_flag#34]), [customer_preferred_cust_flag#34], 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34]
+
+(65) ColumnarToRow [codegen id : 1]
+Input [1]: [customer_preferred_cust_flag#34]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..3a01bb4dcc
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt
@@ -0,0 +1,67 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [customer_preferred_cust_flag]
+ CometProject [customer_preferred_cust_flag]
+ CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total]
+ CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total]
+ CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total]
+ CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total]
+ CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total]
+ CometFilter [customer_id,year_total]
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #3
+ CometFilter [d_date_sk,d_year]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #4
+ CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ ReusedExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometBroadcastExchange [d_date_sk,d_year] #6
+ CometFilter [d_date_sk,d_year]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange [customer_id,year_total] #7
+ CometFilter [customer_id,year_total]
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #9
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ ReusedExchange [d_date_sk,d_year] #3
+ CometBroadcastExchange [customer_id,year_total] #10
+ CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))]
+ CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #9
+ ReusedExchange [d_date_sk,d_year] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt
new file mode 100644
index 0000000000..c6a8dff14a
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt
@@ -0,0 +1,116 @@
+== Physical Plan ==
+TakeOrderedAndProject (22)
++- * Project (21)
+ +- Window (20)
+ +- * ColumnarToRow (19)
+ +- CometSort (18)
+ +- CometExchange (17)
+ +- CometHashAggregate (16)
+ +- CometExchange (15)
+ +- CometHashAggregate (14)
+ +- CometProject (13)
+ +- CometBroadcastHashJoin (12)
+ :- CometProject (7)
+ : +- CometBroadcastHashJoin (6)
+ : :- CometFilter (2)
+ : : +- CometNativeScan: `spark_catalog`.`default`.`web_sales` (1)
+ : +- CometBroadcastExchange (5)
+ : +- CometFilter (4)
+ : +- CometNativeScan: `spark_catalog`.`default`.`item` (3)
+ +- CometBroadcastExchange (11)
+ +- CometProject (10)
+ +- CometFilter (9)
+ +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (8)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`web_sales`
+Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Arguments: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+
+(2) CometFilter
+Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Condition : isnotnull(ws_item_sk#1)
+
+(3) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(4) CometFilter
+Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Condition : (i_category#9 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#4))
+
+(5) CometBroadcastExchange
+Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Right output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [ws_item_sk#1], [i_item_sk#4], Inner, BuildRight
+
+(7) CometProject
+Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(8) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#10, d_date#11]
+Arguments: [d_date_sk#10, d_date#11]
+
+(9) CometFilter
+Input [2]: [d_date_sk#10, d_date#11]
+Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 1999-02-22)) AND (d_date#11 <= 1999-03-24)) AND isnotnull(d_date_sk#10))
+
+(10) CometProject
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: [d_date_sk#10], [d_date_sk#10]
+
+(11) CometBroadcastExchange
+Input [1]: [d_date_sk#10]
+Arguments: [d_date_sk#10]
+
+(12) CometBroadcastHashJoin
+Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Right output [1]: [d_date_sk#10]
+Arguments: [ws_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight
+
+(13) CometProject
+Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9, d_date_sk#10]
+Arguments: [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9], [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(14) CometHashAggregate
+Input [6]: [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))]
+
+(15) CometExchange
+Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#12]
+Arguments: hashpartitioning(i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(16) CometHashAggregate
+Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#12]
+Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))]
+
+(17) CometExchange
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: hashpartitioning(i_class#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(18) CometSort
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5], [i_class#8 ASC NULLS FIRST]
+
+(19) ColumnarToRow [codegen id : 1]
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+
+(20) Window
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: [sum(_w0#14) windowspecdefinition(i_class#8, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#15], [i_class#8]
+
+(21) Project [codegen id : 2]
+Output [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, ((_w0#14 * 100) / _we0#15) AS revenueratio#16, i_item_id#5]
+Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5, _we0#15]
+
+(22) TakeOrderedAndProject
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, revenueratio#16, i_item_id#5]
+Arguments: 100, [i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#16 ASC NULLS FIRST], [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, revenueratio#16]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..64d31a68c7
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt
@@ -0,0 +1,26 @@
+TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue]
+ WholeStageCodegen (2)
+ Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id]
+ InputAdapter
+ Window [_w0,i_class]
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id]
+ CometExchange [i_class] #1
+ CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))]
+ CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
+ CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`web_sales` [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #3
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastExchange [d_date_sk] #4
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_date]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..5fec3af520
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt
@@ -0,0 +1,126 @@
+== Physical Plan ==
+TakeOrderedAndProject (22)
++- * Project (21)
+ +- Window (20)
+ +- * ColumnarToRow (19)
+ +- CometSort (18)
+ +- CometExchange (17)
+ +- CometHashAggregate (16)
+ +- CometExchange (15)
+ +- CometHashAggregate (14)
+ +- CometProject (13)
+ +- CometBroadcastHashJoin (12)
+ :- CometProject (7)
+ : +- CometBroadcastHashJoin (6)
+ : :- CometFilter (2)
+ : : +- CometScan parquet spark_catalog.default.web_sales (1)
+ : +- CometBroadcastExchange (5)
+ : +- CometFilter (4)
+ : +- CometScan parquet spark_catalog.default.item (3)
+ +- CometBroadcastExchange (11)
+ +- CometProject (10)
+ +- CometFilter (9)
+ +- CometScan parquet spark_catalog.default.date_dim (8)
+
+
+(1) CometScan parquet spark_catalog.default.web_sales
+Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#3)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Condition : isnotnull(ws_item_sk#1)
+
+(3) CometScan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Condition : (i_category#9 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#4))
+
+(5) CometBroadcastExchange
+Input [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Right output [6]: [i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [ws_item_sk#1], [i_item_sk#4], Inner, BuildRight
+
+(7) CometProject
+Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#4, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(8) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#10, d_date#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [d_date_sk#10, d_date#11]
+Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 1999-02-22)) AND (d_date#11 <= 1999-03-24)) AND isnotnull(d_date_sk#10))
+
+(10) CometProject
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: [d_date_sk#10], [d_date_sk#10]
+
+(11) CometBroadcastExchange
+Input [1]: [d_date_sk#10]
+Arguments: [d_date_sk#10]
+
+(12) CometBroadcastHashJoin
+Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Right output [1]: [d_date_sk#10]
+Arguments: [ws_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight
+
+(13) CometProject
+Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9, d_date_sk#10]
+Arguments: [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9], [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+
+(14) CometHashAggregate
+Input [6]: [ws_ext_sales_price#2, i_item_id#5, i_item_desc#6, i_current_price#7, i_class#8, i_category#9]
+Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))]
+
+(15) CometExchange
+Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#12]
+Arguments: hashpartitioning(i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(16) CometHashAggregate
+Input [6]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7, sum#12]
+Keys [5]: [i_item_id#5, i_item_desc#6, i_category#9, i_class#8, i_current_price#7]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))]
+
+(17) CometExchange
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: hashpartitioning(i_class#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(18) CometSort
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5], [i_class#8 ASC NULLS FIRST]
+
+(19) ColumnarToRow [codegen id : 1]
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+
+(20) Window
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5]
+Arguments: [sum(_w0#14) windowspecdefinition(i_class#8, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#15], [i_class#8]
+
+(21) Project [codegen id : 2]
+Output [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, ((_w0#14 * 100) / _we0#15) AS revenueratio#16, i_item_id#5]
+Input [8]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, _w0#14, i_item_id#5, _we0#15]
+
+(22) TakeOrderedAndProject
+Input [7]: [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, revenueratio#16, i_item_id#5]
+Arguments: 100, [i_category#9 ASC NULLS FIRST, i_class#8 ASC NULLS FIRST, i_item_id#5 ASC NULLS FIRST, i_item_desc#6 ASC NULLS FIRST, revenueratio#16 ASC NULLS FIRST], [i_item_desc#6, i_category#9, i_class#8, i_current_price#7, itemrevenue#13, revenueratio#16]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..7696b4f605
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt
@@ -0,0 +1,26 @@
+TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue]
+ WholeStageCodegen (2)
+ Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id]
+ InputAdapter
+ Window [_w0,i_class]
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id]
+ CometExchange [i_class] #1
+ CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))]
+ CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
+ CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #3
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastExchange [d_date_sk] #4
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_date]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt
new file mode 100644
index 0000000000..3e48fad3a2
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt
@@ -0,0 +1,174 @@
+== Physical Plan ==
+* ColumnarToRow (33)
++- CometHashAggregate (32)
+ +- CometExchange (31)
+ +- CometHashAggregate (30)
+ +- CometProject (29)
+ +- CometBroadcastHashJoin (28)
+ :- CometProject (24)
+ : +- CometBroadcastHashJoin (23)
+ : :- CometProject (19)
+ : : +- CometBroadcastHashJoin (18)
+ : : :- CometProject (13)
+ : : : +- CometBroadcastHashJoin (12)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`store` (3)
+ : : : +- CometBroadcastExchange (11)
+ : : : +- CometProject (10)
+ : : : +- CometFilter (9)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`customer_address` (8)
+ : : +- CometBroadcastExchange (17)
+ : : +- CometProject (16)
+ : : +- CometFilter (15)
+ : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (14)
+ : +- CometBroadcastExchange (22)
+ : +- CometFilter (21)
+ : +- CometNativeScan: `spark_catalog`.`default`.`customer_demographics` (20)
+ +- CometBroadcastExchange (27)
+ +- CometFilter (26)
+ +- CometNativeScan: `spark_catalog`.`default`.`household_demographics` (25)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+
+(2) CometFilter
+Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00))))
+
+(3) CometNativeScan: `spark_catalog`.`default`.`store`
+Output [1]: [s_store_sk#11]
+Arguments: [s_store_sk#11]
+
+(4) CometFilter
+Input [1]: [s_store_sk#11]
+Condition : isnotnull(s_store_sk#11)
+
+(5) CometBroadcastExchange
+Input [1]: [s_store_sk#11]
+Arguments: [s_store_sk#11]
+
+(6) CometBroadcastHashJoin
+Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [1]: [s_store_sk#11]
+Arguments: [ss_store_sk#4], [s_store_sk#11], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#11]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+
+(8) CometNativeScan: `spark_catalog`.`default`.`customer_address`
+Output [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Arguments: [ca_address_sk#12, ca_state#13, ca_country#14]
+
+(9) CometFilter
+Input [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Condition : (((isnotnull(ca_country#14) AND (ca_country#14 = United States)) AND isnotnull(ca_address_sk#12)) AND ((ca_state#13 IN (TX,OH) OR ca_state#13 IN (OR,NM,KY)) OR ca_state#13 IN (VA,TX,MS)))
+
+(10) CometProject
+Input [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Arguments: [ca_address_sk#12, ca_state#13], [ca_address_sk#12, ca_state#13]
+
+(11) CometBroadcastExchange
+Input [2]: [ca_address_sk#12, ca_state#13]
+Arguments: [ca_address_sk#12, ca_state#13]
+
+(12) CometBroadcastHashJoin
+Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [2]: [ca_address_sk#12, ca_state#13]
+Arguments: [ss_addr_sk#3], [ca_address_sk#12], Inner, ((((ca_state#13 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#13 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#13 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight
+
+(13) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#12, ca_state#13]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+
+(14) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#15, d_year#16]
+Arguments: [d_date_sk#15, d_year#16]
+
+(15) CometFilter
+Input [2]: [d_date_sk#15, d_year#16]
+Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_sk#15))
+
+(16) CometProject
+Input [2]: [d_date_sk#15, d_year#16]
+Arguments: [d_date_sk#15], [d_date_sk#15]
+
+(17) CometBroadcastExchange
+Input [1]: [d_date_sk#15]
+Arguments: [d_date_sk#15]
+
+(18) CometBroadcastHashJoin
+Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+Right output [1]: [d_date_sk#15]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#15], Inner, BuildRight
+
+(19) CometProject
+Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#15]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(20) CometNativeScan: `spark_catalog`.`default`.`customer_demographics`
+Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+
+(21) CometFilter
+Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree ))))
+
+(22) CometBroadcastExchange
+Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+
+(23) CometBroadcastHashJoin
+Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Right output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [ss_cdemo_sk#1], [cd_demo_sk#17], Inner, ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight
+
+(24) CometProject
+Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19]
+
+(25) CometNativeScan: `spark_catalog`.`default`.`household_demographics`
+Output [2]: [hd_demo_sk#20, hd_dep_count#21]
+Arguments: [hd_demo_sk#20, hd_dep_count#21]
+
+(26) CometFilter
+Input [2]: [hd_demo_sk#20, hd_dep_count#21]
+Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1)))
+
+(27) CometBroadcastExchange
+Input [2]: [hd_demo_sk#20, hd_dep_count#21]
+Arguments: [hd_demo_sk#20, hd_dep_count#21]
+
+(28) CometBroadcastHashJoin
+Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19]
+Right output [2]: [hd_demo_sk#20, hd_dep_count#21]
+Arguments: [ss_hdemo_sk#2], [hd_demo_sk#20], Inner, (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))), BuildRight
+
+(29) CometProject
+Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21]
+Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(30) CometHashAggregate
+Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Keys: []
+Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+
+(31) CometExchange
+Input [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28]
+Keys: []
+Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+
+(33) ColumnarToRow [codegen id : 1]
+Input [4]: [avg(ss_quantity)#29, avg(ss_ext_sales_price)#30, avg(ss_ext_wholesale_cost)#31, sum(ss_ext_wholesale_cost)#32]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..ec41eb52c3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt
@@ -0,0 +1,35 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))]
+ CometExchange #1
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count]
+ CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometBroadcastExchange [s_store_sk] #2
+ CometFilter [s_store_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store` [s_store_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state] #3
+ CometProject [ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
+ CometNativeScan: `spark_catalog`.`default`.`customer_address` [ca_address_sk,ca_state,ca_country]
+ CometBroadcastExchange [d_date_sk] #4
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5
+ CometFilter [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometNativeScan: `spark_catalog`.`default`.`customer_demographics` [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastExchange [hd_demo_sk,hd_dep_count] #6
+ CometFilter [hd_demo_sk,hd_dep_count]
+ CometNativeScan: `spark_catalog`.`default`.`household_demographics` [hd_demo_sk,hd_dep_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..8dfa77e2f6
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt
@@ -0,0 +1,193 @@
+== Physical Plan ==
+* ColumnarToRow (33)
++- CometHashAggregate (32)
+ +- CometExchange (31)
+ +- CometHashAggregate (30)
+ +- CometProject (29)
+ +- CometBroadcastHashJoin (28)
+ :- CometProject (24)
+ : +- CometBroadcastHashJoin (23)
+ : :- CometProject (19)
+ : : +- CometBroadcastHashJoin (18)
+ : : :- CometProject (13)
+ : : : +- CometBroadcastHashJoin (12)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.store (3)
+ : : : +- CometBroadcastExchange (11)
+ : : : +- CometProject (10)
+ : : : +- CometFilter (9)
+ : : : +- CometScan parquet spark_catalog.default.customer_address (8)
+ : : +- CometBroadcastExchange (17)
+ : : +- CometProject (16)
+ : : +- CometFilter (15)
+ : : +- CometScan parquet spark_catalog.default.date_dim (14)
+ : +- CometBroadcastExchange (22)
+ : +- CometFilter (21)
+ : +- CometScan parquet spark_catalog.default.customer_demographics (20)
+ +- CometBroadcastExchange (27)
+ +- CometFilter (26)
+ +- CometScan parquet spark_catalog.default.household_demographics (25)
+
+
+(1) CometScan parquet spark_catalog.default.store_sales
+Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#10)]
+PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))]
+ReadSchema: struct
+
+(2) CometFilter
+Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00))))
+
+(3) CometScan parquet spark_catalog.default.store
+Output [1]: [s_store_sk#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [1]: [s_store_sk#11]
+Condition : isnotnull(s_store_sk#11)
+
+(5) CometBroadcastExchange
+Input [1]: [s_store_sk#11]
+Arguments: [s_store_sk#11]
+
+(6) CometBroadcastHashJoin
+Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [1]: [s_store_sk#11]
+Arguments: [ss_store_sk#4], [s_store_sk#11], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#11]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+
+(8) CometScan parquet spark_catalog.default.customer_address
+Output [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))]
+ReadSchema: struct
+
+(9) CometFilter
+Input [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Condition : (((isnotnull(ca_country#14) AND (ca_country#14 = United States)) AND isnotnull(ca_address_sk#12)) AND ((ca_state#13 IN (TX,OH) OR ca_state#13 IN (OR,NM,KY)) OR ca_state#13 IN (VA,TX,MS)))
+
+(10) CometProject
+Input [3]: [ca_address_sk#12, ca_state#13, ca_country#14]
+Arguments: [ca_address_sk#12, ca_state#13], [ca_address_sk#12, ca_state#13]
+
+(11) CometBroadcastExchange
+Input [2]: [ca_address_sk#12, ca_state#13]
+Arguments: [ca_address_sk#12, ca_state#13]
+
+(12) CometBroadcastHashJoin
+Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [2]: [ca_address_sk#12, ca_state#13]
+Arguments: [ss_addr_sk#3], [ca_address_sk#12], Inner, ((((ca_state#13 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#13 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#13 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight
+
+(13) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#12, ca_state#13]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+
+(14) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#15, d_year#16]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(15) CometFilter
+Input [2]: [d_date_sk#15, d_year#16]
+Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_sk#15))
+
+(16) CometProject
+Input [2]: [d_date_sk#15, d_year#16]
+Arguments: [d_date_sk#15], [d_date_sk#15]
+
+(17) CometBroadcastExchange
+Input [1]: [d_date_sk#15]
+Arguments: [d_date_sk#15]
+
+(18) CometBroadcastHashJoin
+Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+Right output [1]: [d_date_sk#15]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#15], Inner, BuildRight
+
+(19) CometProject
+Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#15]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(20) CometScan parquet spark_catalog.default.customer_demographics
+Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))]
+ReadSchema: struct
+
+(21) CometFilter
+Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree ))))
+
+(22) CometBroadcastExchange
+Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+
+(23) CometBroadcastHashJoin
+Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Right output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [ss_cdemo_sk#1], [cd_demo_sk#17], Inner, ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight
+
+(24) CometProject
+Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19]
+Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19]
+
+(25) CometScan parquet spark_catalog.default.household_demographics
+Output [2]: [hd_demo_sk#20, hd_dep_count#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/household_demographics]
+PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [hd_demo_sk#20, hd_dep_count#21]
+Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1)))
+
+(27) CometBroadcastExchange
+Input [2]: [hd_demo_sk#20, hd_dep_count#21]
+Arguments: [hd_demo_sk#20, hd_dep_count#21]
+
+(28) CometBroadcastHashJoin
+Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19]
+Right output [2]: [hd_demo_sk#20, hd_dep_count#21]
+Arguments: [ss_hdemo_sk#2], [hd_demo_sk#20], Inner, (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))), BuildRight
+
+(29) CometProject
+Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21]
+Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(30) CometHashAggregate
+Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Keys: []
+Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+
+(31) CometExchange
+Input [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28]
+Keys: []
+Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+
+(33) ColumnarToRow [codegen id : 1]
+Input [4]: [avg(ss_quantity)#29, avg(ss_ext_sales_price)#30, avg(ss_ext_wholesale_cost)#31, sum(ss_ext_wholesale_cost)#32]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..dc2f87896a
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt
@@ -0,0 +1,35 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum,avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))]
+ CometExchange #1
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count]
+ CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometBroadcastExchange [s_store_sk] #2
+ CometFilter [s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state] #3
+ CometProject [ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
+ CometBroadcastExchange [d_date_sk] #4
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5
+ CometFilter [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastExchange [hd_demo_sk,hd_dep_count] #6
+ CometFilter [hd_demo_sk,hd_dep_count]
+ CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt
new file mode 100644
index 0000000000..c271ba7ee3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt
@@ -0,0 +1,474 @@
+== Physical Plan ==
+* ColumnarToRow (72)
++- CometTakeOrderedAndProject (71)
+ +- CometHashAggregate (70)
+ +- CometExchange (69)
+ +- CometHashAggregate (68)
+ +- CometExpand (67)
+ +- CometUnion (66)
+ :- CometProject (57)
+ : +- CometFilter (56)
+ : +- CometHashAggregate (55)
+ : +- CometExchange (54)
+ : +- CometHashAggregate (53)
+ : +- CometProject (52)
+ : +- CometBroadcastHashJoin (51)
+ : :- CometProject (46)
+ : : +- CometBroadcastHashJoin (45)
+ : : :- CometBroadcastHashJoin (39)
+ : : : :- CometFilter (2)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (1)
+ : : : +- CometBroadcastExchange (38)
+ : : : +- CometProject (37)
+ : : : +- CometBroadcastHashJoin (36)
+ : : : :- CometFilter (4)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (3)
+ : : : +- CometBroadcastExchange (35)
+ : : : +- CometBroadcastHashJoin (34)
+ : : : :- CometHashAggregate (32)
+ : : : : +- CometExchange (31)
+ : : : : +- CometHashAggregate (30)
+ : : : : +- CometProject (29)
+ : : : : +- CometBroadcastHashJoin (28)
+ : : : : :- CometProject (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (6)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (5)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometBroadcastHashJoin (23)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (7)
+ : : : : : +- CometBroadcastExchange (22)
+ : : : : : +- CometProject (21)
+ : : : : : +- CometBroadcastHashJoin (20)
+ : : : : : :- CometProject (15)
+ : : : : : : +- CometBroadcastHashJoin (14)
+ : : : : : : :- CometFilter (10)
+ : : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`catalog_sales` (9)
+ : : : : : : +- CometBroadcastExchange (13)
+ : : : : : : +- CometFilter (12)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (11)
+ : : : : : +- CometBroadcastExchange (19)
+ : : : : : +- CometProject (18)
+ : : : : : +- CometFilter (17)
+ : : : : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (16)
+ : : : : +- ReusedExchange (27)
+ : : : +- ReusedExchange (33)
+ : : +- CometBroadcastExchange (44)
+ : : +- CometBroadcastHashJoin (43)
+ : : :- CometFilter (41)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (40)
+ : : +- ReusedExchange (42)
+ : +- CometBroadcastExchange (50)
+ : +- CometProject (49)
+ : +- CometFilter (48)
+ : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (47)
+ :- CometProject (61)
+ : +- CometFilter (60)
+ : +- CometHashAggregate (59)
+ : +- ReusedExchange (58)
+ +- CometProject (65)
+ +- CometFilter (64)
+ +- CometHashAggregate (63)
+ +- ReusedExchange (62)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Arguments: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Arguments: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+
+(4) CometFilter
+Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8))
+
+(5) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Arguments: [ss_item_sk#9, ss_sold_date_sk#10]
+
+(6) CometFilter
+Input [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Condition : isnotnull(ss_item_sk#9)
+
+(7) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(8) CometFilter
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14))
+
+(9) CometNativeScan: `spark_catalog`.`default`.`catalog_sales`
+Output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Arguments: [cs_item_sk#15, cs_sold_date_sk#16]
+
+(10) CometFilter
+Input [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Condition : isnotnull(cs_item_sk#15)
+
+(11) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(12) CometFilter
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Condition : isnotnull(i_item_sk#17)
+
+(13) CometBroadcastExchange
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(14) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Right output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_item_sk#15], [i_item_sk#17], Inner, BuildRight
+
+(15) CometProject
+Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20], [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(16) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21, d_year#22]
+
+(17) CometFilter
+Input [2]: [d_date_sk#21, d_year#22]
+Condition : (((isnotnull(d_year#22) AND (d_year#22 >= 1999)) AND (d_year#22 <= 2001)) AND isnotnull(d_date_sk#21))
+
+(18) CometProject
+Input [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(19) CometBroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: [d_date_sk#21]
+
+(20) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+Right output [1]: [d_date_sk#21]
+Arguments: [cs_sold_date_sk#16], [d_date_sk#21], Inner, BuildRight
+
+(21) CometProject
+Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#21]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20], [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(22) CometBroadcastExchange
+Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(23) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)], [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)], LeftSemi, BuildRight
+
+(24) CometBroadcastExchange
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(25) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Right output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_item_sk#9], [i_item_sk#11], Inner, BuildRight
+
+(26) CometProject
+Input [6]: [ss_item_sk#9, ss_sold_date_sk#10, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14], [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(27) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#23]
+
+(28) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [1]: [d_date_sk#23]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#23], Inner, BuildRight
+
+(29) CometProject
+Input [5]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#23]
+Arguments: [brand_id#24, class_id#25, category_id#26], [i_brand_id#12 AS brand_id#24, i_class_id#13 AS class_id#25, i_category_id#14 AS category_id#26]
+
+(30) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(31) CometExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: hashpartitioning(brand_id#24, class_id#25, category_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(33) ReusedExchange [Reuses operator id: 22]
+Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29]
+
+(34) CometBroadcastHashJoin
+Left output [3]: [brand_id#24, class_id#25, category_id#26]
+Right output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29]
+Arguments: [coalesce(brand_id#24, 0), isnull(brand_id#24), coalesce(class_id#25, 0), isnull(class_id#25), coalesce(category_id#26, 0), isnull(category_id#26)], [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)], LeftSemi, BuildRight
+
+(35) CometBroadcastExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [brand_id#24, class_id#25, category_id#26]
+
+(36) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Right output [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [i_brand_id#6, i_class_id#7, i_category_id#8], [brand_id#24, class_id#25, category_id#26], Inner, BuildRight
+
+(37) CometProject
+Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#24, class_id#25, category_id#26]
+Arguments: [ss_item_sk#30], [i_item_sk#5 AS ss_item_sk#30]
+
+(38) CometBroadcastExchange
+Input [1]: [ss_item_sk#30]
+Arguments: [ss_item_sk#30]
+
+(39) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [1]: [ss_item_sk#30]
+Arguments: [ss_item_sk#1], [ss_item_sk#30], LeftSemi, BuildRight
+
+(40) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(41) CometFilter
+Input [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Condition : isnotnull(i_item_sk#31)
+
+(42) ReusedExchange [Reuses operator id: 38]
+Output [1]: [ss_item_sk#30]
+
+(43) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Right output [1]: [ss_item_sk#30]
+Arguments: [i_item_sk#31], [ss_item_sk#30], LeftSemi, BuildRight
+
+(44) CometBroadcastExchange
+Input [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(45) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight
+
+(46) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(47) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [3]: [d_date_sk#35, d_year#36, d_moy#37]
+Arguments: [d_date_sk#35, d_year#36, d_moy#37]
+
+(48) CometFilter
+Input [3]: [d_date_sk#35, d_year#36, d_moy#37]
+Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 11)) AND isnotnull(d_date_sk#35))
+
+(49) CometProject
+Input [3]: [d_date_sk#35, d_year#36, d_moy#37]
+Arguments: [d_date_sk#35], [d_date_sk#35]
+
+(50) CometBroadcastExchange
+Input [1]: [d_date_sk#35]
+Arguments: [d_date_sk#35]
+
+(51) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34]
+Right output [1]: [d_date_sk#35]
+Arguments: [ss_sold_date_sk#4], [d_date_sk#35], Inner, BuildRight
+
+(52) CometProject
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34, d_date_sk#35]
+Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34], [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(53) CometHashAggregate
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34]
+Keys [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+
+(54) CometExchange
+Input [6]: [i_brand_id#32, i_class_id#33, i_category_id#34, sum#38, isEmpty#39, count#40]
+Arguments: hashpartitioning(i_brand_id#32, i_class_id#33, i_category_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(55) CometHashAggregate
+Input [6]: [i_brand_id#32, i_class_id#33, i_category_id#34, sum#38, isEmpty#39, count#40]
+Keys [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+
+(56) CometFilter
+Input [5]: [i_brand_id#32, i_class_id#33, i_category_id#34, sales#41, number_sales#42]
+Condition : (isnotnull(sales#41) AND (cast(sales#41 as decimal(32,6)) > cast(Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
+
+(57) CometProject
+Input [5]: [i_brand_id#32, i_class_id#33, i_category_id#34, sales#41, number_sales#42]
+Arguments: [sales#41, number_sales#42, channel#45, i_brand_id#46, i_class_id#47, i_category_id#48], [sales#41, number_sales#42, store AS channel#45, i_brand_id#32 AS i_brand_id#46, i_class_id#33 AS i_class_id#47, i_category_id#34 AS i_category_id#48]
+
+(58) ReusedExchange [Reuses operator id: 54]
+Output [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#52, isEmpty#53, count#54]
+
+(59) CometHashAggregate
+Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#52, isEmpty#53, count#54]
+Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51]
+Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
+
+(60) CometFilter
+Input [5]: [i_brand_id#49, i_class_id#50, i_category_id#51, sales#57, number_sales#58]
+Condition : (isnotnull(sales#57) AND (cast(sales#57 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
+
+(61) CometProject
+Input [5]: [i_brand_id#49, i_class_id#50, i_category_id#51, sales#57, number_sales#58]
+Arguments: [sales#57, number_sales#58, channel#59, i_brand_id#49, i_class_id#50, i_category_id#51], [sales#57, number_sales#58, catalog AS channel#59, i_brand_id#49, i_class_id#50, i_category_id#51]
+
+(62) ReusedExchange [Reuses operator id: 54]
+Output [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#63, isEmpty#64, count#65]
+
+(63) CometHashAggregate
+Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#63, isEmpty#64, count#65]
+Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
+Functions [2]: [sum((cast(ws_quantity#66 as decimal(10,0)) * ws_list_price#67)), count(1)]
+
+(64) CometFilter
+Input [5]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69]
+Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
+
+(65) CometProject
+Input [5]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69]
+Arguments: [sales#68, number_sales#69, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62], [sales#68, number_sales#69, web AS channel#70, i_brand_id#60, i_class_id#61, i_category_id#62]
+
+(66) CometUnion
+Child 0 Input [6]: [sales#41, number_sales#42, channel#45, i_brand_id#46, i_class_id#47, i_category_id#48]
+Child 1 Input [6]: [sales#57, number_sales#58, channel#59, i_brand_id#49, i_class_id#50, i_category_id#51]
+Child 2 Input [6]: [sales#68, number_sales#69, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62]
+
+(67) CometExpand
+Input [6]: [sales#41, number_sales#42, channel#45, i_brand_id#46, i_class_id#47, i_category_id#48]
+Arguments: [[sales#41, number_sales#42, channel#45, i_brand_id#46, i_class_id#47, i_category_id#48, 0], [sales#41, number_sales#42, channel#45, i_brand_id#46, i_class_id#47, null, 1], [sales#41, number_sales#42, channel#45, i_brand_id#46, null, null, 3], [sales#41, number_sales#42, channel#45, null, null, null, 7], [sales#41, number_sales#42, null, null, null, null, 15]], [sales#41, number_sales#42, channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75]
+
+(68) CometHashAggregate
+Input [7]: [sales#41, number_sales#42, channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75]
+Keys [5]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75]
+Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)]
+
+(69) CometExchange
+Input [8]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75, sum#76, isEmpty#77, sum#78]
+Arguments: hashpartitioning(channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(70) CometHashAggregate
+Input [8]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75, sum#76, isEmpty#77, sum#78]
+Keys [5]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, spark_grouping_id#75]
+Functions [2]: [sum(sales#41), sum(number_sales#42)]
+
+(71) CometTakeOrderedAndProject
+Input [6]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, sum(sales)#79, sum(number_sales)#80]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#71 ASC NULLS FIRST,i_brand_id#72 ASC NULLS FIRST,i_class_id#73 ASC NULLS FIRST,i_category_id#74 ASC NULLS FIRST], output=[channel#71,i_brand_id#72,i_class_id#73,i_category_id#74,sum(sales)#79,sum(number_sales)#80]), [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, sum(sales)#79, sum(number_sales)#80], 100, [channel#71 ASC NULLS FIRST, i_brand_id#72 ASC NULLS FIRST, i_class_id#73 ASC NULLS FIRST, i_category_id#74 ASC NULLS FIRST], [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, sum(sales)#79, sum(number_sales)#80]
+
+(72) ColumnarToRow [codegen id : 1]
+Input [6]: [channel#71, i_brand_id#72, i_class_id#73, i_category_id#74, sum(sales)#79, sum(number_sales)#80]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#43, [id=#44]
+* ColumnarToRow (89)
++- CometHashAggregate (88)
+ +- CometExchange (87)
+ +- CometHashAggregate (86)
+ +- CometUnion (85)
+ :- CometProject (76)
+ : +- CometBroadcastHashJoin (75)
+ : :- CometNativeScan: `spark_catalog`.`default`.`store_sales` (73)
+ : +- ReusedExchange (74)
+ :- CometProject (80)
+ : +- CometBroadcastHashJoin (79)
+ : :- CometNativeScan: `spark_catalog`.`default`.`catalog_sales` (77)
+ : +- ReusedExchange (78)
+ +- CometProject (84)
+ +- CometBroadcastHashJoin (83)
+ :- CometNativeScan: `spark_catalog`.`default`.`web_sales` (81)
+ +- ReusedExchange (82)
+
+
+(73) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [3]: [ss_quantity#81, ss_list_price#82, ss_sold_date_sk#83]
+Arguments: [ss_quantity#81, ss_list_price#82, ss_sold_date_sk#83]
+
+(74) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#84]
+
+(75) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#81, ss_list_price#82, ss_sold_date_sk#83]
+Right output [1]: [d_date_sk#84]
+Arguments: [ss_sold_date_sk#83], [d_date_sk#84], Inner, BuildRight
+
+(76) CometProject
+Input [4]: [ss_quantity#81, ss_list_price#82, ss_sold_date_sk#83, d_date_sk#84]
+Arguments: [quantity#85, list_price#86], [ss_quantity#81 AS quantity#85, ss_list_price#82 AS list_price#86]
+
+(77) CometNativeScan: `spark_catalog`.`default`.`catalog_sales`
+Output [3]: [cs_quantity#87, cs_list_price#88, cs_sold_date_sk#89]
+Arguments: [cs_quantity#87, cs_list_price#88, cs_sold_date_sk#89]
+
+(78) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#90]
+
+(79) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#87, cs_list_price#88, cs_sold_date_sk#89]
+Right output [1]: [d_date_sk#90]
+Arguments: [cs_sold_date_sk#89], [d_date_sk#90], Inner, BuildRight
+
+(80) CometProject
+Input [4]: [cs_quantity#87, cs_list_price#88, cs_sold_date_sk#89, d_date_sk#90]
+Arguments: [quantity#91, list_price#92], [cs_quantity#87 AS quantity#91, cs_list_price#88 AS list_price#92]
+
+(81) CometNativeScan: `spark_catalog`.`default`.`web_sales`
+Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95]
+Arguments: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95]
+
+(82) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#96]
+
+(83) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95]
+Right output [1]: [d_date_sk#96]
+Arguments: [ws_sold_date_sk#95], [d_date_sk#96], Inner, BuildRight
+
+(84) CometProject
+Input [4]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95, d_date_sk#96]
+Arguments: [quantity#97, list_price#98], [ws_quantity#93 AS quantity#97, ws_list_price#94 AS list_price#98]
+
+(85) CometUnion
+Child 0 Input [2]: [quantity#85, list_price#86]
+Child 1 Input [2]: [quantity#91, list_price#92]
+Child 2 Input [2]: [quantity#97, list_price#98]
+
+(86) CometHashAggregate
+Input [2]: [quantity#85, list_price#86]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#85 as decimal(10,0)) * list_price#86))]
+
+(87) CometExchange
+Input [2]: [sum#99, count#100]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(88) CometHashAggregate
+Input [2]: [sum#99, count#100]
+Keys: []
+Functions [1]: [avg((cast(quantity#85 as decimal(10,0)) * list_price#86))]
+
+(89) ColumnarToRow [codegen id : 1]
+Input [1]: [average_sales#101]
+
+Subquery:2 Hosting operator id = 60 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
+
+Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..12407d32a3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt
@@ -0,0 +1,96 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)]
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)]
+ CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales]
+ CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id]
+ CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ Subquery #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))]
+ CometExchange #12
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
+ CometProject [ss_quantity,ss_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometProject [cs_quantity,cs_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`catalog_sales` [cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometProject [ws_quantity,ws_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`web_sales` [ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #2
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price]
+ CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometBroadcastExchange [ss_item_sk] #3
+ CometProject [i_item_sk] [ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [brand_id,class_id,category_id] #4
+ CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometExchange [brand_id,class_id,category_id] #5
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_item_sk,ss_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #7
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`catalog_sales` [cs_item_sk,cs_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [d_date_sk] #9
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ ReusedExchange [d_date_sk] #9
+ ReusedExchange [i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ ReusedExchange [ss_item_sk] #3
+ CometBroadcastExchange [d_date_sk] #11
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year,d_moy]
+ CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ ReusedSubquery [average_sales] #1
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)]
+ ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2
+ CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ ReusedSubquery [average_sales] #1
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)]
+ ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #2
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..38a0d1d009
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt
@@ -0,0 +1,680 @@
+== Physical Plan ==
+* ColumnarToRow (102)
++- CometTakeOrderedAndProject (101)
+ +- CometHashAggregate (100)
+ +- CometExchange (99)
+ +- CometHashAggregate (98)
+ +- CometExpand (97)
+ +- CometUnion (96)
+ :- CometProject (65)
+ : +- CometFilter (64)
+ : +- CometHashAggregate (63)
+ : +- CometExchange (62)
+ : +- CometHashAggregate (61)
+ : +- CometProject (60)
+ : +- CometBroadcastHashJoin (59)
+ : :- CometProject (54)
+ : : +- CometBroadcastHashJoin (53)
+ : : :- CometBroadcastHashJoin (47)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- CometBroadcastExchange (46)
+ : : : +- CometProject (45)
+ : : : +- CometBroadcastHashJoin (44)
+ : : : :- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.item (3)
+ : : : +- CometBroadcastExchange (43)
+ : : : +- CometBroadcastHashJoin (42)
+ : : : :- CometHashAggregate (32)
+ : : : : +- CometExchange (31)
+ : : : : +- CometHashAggregate (30)
+ : : : : +- CometProject (29)
+ : : : : +- CometBroadcastHashJoin (28)
+ : : : : :- CometProject (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (6)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometBroadcastHashJoin (23)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (7)
+ : : : : : +- CometBroadcastExchange (22)
+ : : : : : +- CometProject (21)
+ : : : : : +- CometBroadcastHashJoin (20)
+ : : : : : :- CometProject (15)
+ : : : : : : +- CometBroadcastHashJoin (14)
+ : : : : : : :- CometFilter (10)
+ : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9)
+ : : : : : : +- CometBroadcastExchange (13)
+ : : : : : : +- CometFilter (12)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (11)
+ : : : : : +- CometBroadcastExchange (19)
+ : : : : : +- CometProject (18)
+ : : : : : +- CometFilter (17)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (16)
+ : : : : +- ReusedExchange (27)
+ : : : +- CometBroadcastExchange (41)
+ : : : +- CometProject (40)
+ : : : +- CometBroadcastHashJoin (39)
+ : : : :- CometProject (37)
+ : : : : +- CometBroadcastHashJoin (36)
+ : : : : :- CometFilter (34)
+ : : : : : +- CometScan parquet spark_catalog.default.web_sales (33)
+ : : : : +- ReusedExchange (35)
+ : : : +- ReusedExchange (38)
+ : : +- CometBroadcastExchange (52)
+ : : +- CometBroadcastHashJoin (51)
+ : : :- CometFilter (49)
+ : : : +- CometScan parquet spark_catalog.default.item (48)
+ : : +- ReusedExchange (50)
+ : +- CometBroadcastExchange (58)
+ : +- CometProject (57)
+ : +- CometFilter (56)
+ : +- CometScan parquet spark_catalog.default.date_dim (55)
+ :- CometProject (80)
+ : +- CometFilter (79)
+ : +- CometHashAggregate (78)
+ : +- CometExchange (77)
+ : +- CometHashAggregate (76)
+ : +- CometProject (75)
+ : +- CometBroadcastHashJoin (74)
+ : :- CometProject (72)
+ : : +- CometBroadcastHashJoin (71)
+ : : :- CometBroadcastHashJoin (69)
+ : : : :- CometFilter (67)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (66)
+ : : : +- ReusedExchange (68)
+ : : +- ReusedExchange (70)
+ : +- ReusedExchange (73)
+ +- CometProject (95)
+ +- CometFilter (94)
+ +- CometHashAggregate (93)
+ +- CometExchange (92)
+ +- CometHashAggregate (91)
+ +- CometProject (90)
+ +- CometBroadcastHashJoin (89)
+ :- CometProject (87)
+ : +- CometBroadcastHashJoin (86)
+ : :- CometBroadcastHashJoin (84)
+ : : :- CometFilter (82)
+ : : : +- CometScan parquet spark_catalog.default.web_sales (81)
+ : : +- ReusedExchange (83)
+ : +- ReusedExchange (85)
+ +- ReusedExchange (88)
+
+
+(1) CometScan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#4)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8))
+
+(5) CometScan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#10)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(6) CometFilter
+Input [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Condition : isnotnull(ss_item_sk#9)
+
+(7) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14))
+
+(9) CometScan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#16)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Condition : isnotnull(cs_item_sk#15)
+
+(11) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(12) CometFilter
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Condition : isnotnull(i_item_sk#17)
+
+(13) CometBroadcastExchange
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(14) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Right output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_item_sk#15], [i_item_sk#17], Inner, BuildRight
+
+(15) CometProject
+Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20], [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(16) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#21, d_year#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(17) CometFilter
+Input [2]: [d_date_sk#21, d_year#22]
+Condition : (((isnotnull(d_year#22) AND (d_year#22 >= 1999)) AND (d_year#22 <= 2001)) AND isnotnull(d_date_sk#21))
+
+(18) CometProject
+Input [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(19) CometBroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: [d_date_sk#21]
+
+(20) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+Right output [1]: [d_date_sk#21]
+Arguments: [cs_sold_date_sk#16], [d_date_sk#21], Inner, BuildRight
+
+(21) CometProject
+Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#21]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20], [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(22) CometBroadcastExchange
+Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(23) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)], [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)], LeftSemi, BuildRight
+
+(24) CometBroadcastExchange
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(25) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Right output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_item_sk#9], [i_item_sk#11], Inner, BuildRight
+
+(26) CometProject
+Input [6]: [ss_item_sk#9, ss_sold_date_sk#10, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14], [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(27) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#23]
+
+(28) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [1]: [d_date_sk#23]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#23], Inner, BuildRight
+
+(29) CometProject
+Input [5]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#23]
+Arguments: [brand_id#24, class_id#25, category_id#26], [i_brand_id#12 AS brand_id#24, i_class_id#13 AS class_id#25, i_category_id#14 AS category_id#26]
+
+(30) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(31) CometExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: hashpartitioning(brand_id#24, class_id#25, category_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(33) CometScan parquet spark_catalog.default.web_sales
+Output [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#28)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Condition : isnotnull(ws_item_sk#27)
+
+(35) ReusedExchange [Reuses operator id: 13]
+Output [4]: [i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(36) CometBroadcastHashJoin
+Left output [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Right output [4]: [i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [ws_item_sk#27], [i_item_sk#29], Inner, BuildRight
+
+(37) CometProject
+Input [6]: [ws_item_sk#27, ws_sold_date_sk#28, i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32], [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(38) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#33]
+
+(39) CometBroadcastHashJoin
+Left output [4]: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32]
+Right output [1]: [d_date_sk#33]
+Arguments: [ws_sold_date_sk#28], [d_date_sk#33], Inner, BuildRight
+
+(40) CometProject
+Input [5]: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32, d_date_sk#33]
+Arguments: [i_brand_id#30, i_class_id#31, i_category_id#32], [i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(41) CometBroadcastExchange
+Input [3]: [i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(42) CometBroadcastHashJoin
+Left output [3]: [brand_id#24, class_id#25, category_id#26]
+Right output [3]: [i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [coalesce(brand_id#24, 0), isnull(brand_id#24), coalesce(class_id#25, 0), isnull(class_id#25), coalesce(category_id#26, 0), isnull(category_id#26)], [coalesce(i_brand_id#30, 0), isnull(i_brand_id#30), coalesce(i_class_id#31, 0), isnull(i_class_id#31), coalesce(i_category_id#32, 0), isnull(i_category_id#32)], LeftSemi, BuildRight
+
+(43) CometBroadcastExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [brand_id#24, class_id#25, category_id#26]
+
+(44) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Right output [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [i_brand_id#6, i_class_id#7, i_category_id#8], [brand_id#24, class_id#25, category_id#26], Inner, BuildRight
+
+(45) CometProject
+Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#24, class_id#25, category_id#26]
+Arguments: [ss_item_sk#34], [i_item_sk#5 AS ss_item_sk#34]
+
+(46) CometBroadcastExchange
+Input [1]: [ss_item_sk#34]
+Arguments: [ss_item_sk#34]
+
+(47) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [1]: [ss_item_sk#34]
+Arguments: [ss_item_sk#1], [ss_item_sk#34], LeftSemi, BuildRight
+
+(48) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(49) CometFilter
+Input [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Condition : isnotnull(i_item_sk#35)
+
+(50) ReusedExchange [Reuses operator id: 46]
+Output [1]: [ss_item_sk#34]
+
+(51) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Right output [1]: [ss_item_sk#34]
+Arguments: [i_item_sk#35], [ss_item_sk#34], LeftSemi, BuildRight
+
+(52) CometBroadcastExchange
+Input [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(53) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [ss_item_sk#1], [i_item_sk#35], Inner, BuildRight
+
+(54) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(55) CometScan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#39, d_year#40, d_moy#41]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(56) CometFilter
+Input [3]: [d_date_sk#39, d_year#40, d_moy#41]
+Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 11)) AND isnotnull(d_date_sk#39))
+
+(57) CometProject
+Input [3]: [d_date_sk#39, d_year#40, d_moy#41]
+Arguments: [d_date_sk#39], [d_date_sk#39]
+
+(58) CometBroadcastExchange
+Input [1]: [d_date_sk#39]
+Arguments: [d_date_sk#39]
+
+(59) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38]
+Right output [1]: [d_date_sk#39]
+Arguments: [ss_sold_date_sk#4], [d_date_sk#39], Inner, BuildRight
+
+(60) CometProject
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38, d_date_sk#39]
+Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38], [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(61) CometHashAggregate
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38]
+Keys [3]: [i_brand_id#36, i_class_id#37, i_category_id#38]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+
+(62) CometExchange
+Input [6]: [i_brand_id#36, i_class_id#37, i_category_id#38, sum#42, isEmpty#43, count#44]
+Arguments: hashpartitioning(i_brand_id#36, i_class_id#37, i_category_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(63) CometHashAggregate
+Input [6]: [i_brand_id#36, i_class_id#37, i_category_id#38, sum#42, isEmpty#43, count#44]
+Keys [3]: [i_brand_id#36, i_class_id#37, i_category_id#38]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+
+(64) CometFilter
+Input [5]: [i_brand_id#36, i_class_id#37, i_category_id#38, sales#45, number_sales#46]
+Condition : (isnotnull(sales#45) AND (cast(sales#45 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
+
+(65) CometProject
+Input [5]: [i_brand_id#36, i_class_id#37, i_category_id#38, sales#45, number_sales#46]
+Arguments: [sales#45, number_sales#46, channel#49, i_brand_id#50, i_class_id#51, i_category_id#52], [sales#45, number_sales#46, store AS channel#49, i_brand_id#36 AS i_brand_id#50, i_class_id#37 AS i_class_id#51, i_category_id#38 AS i_category_id#52]
+
+(66) CometScan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#56)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(67) CometFilter
+Input [4]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56]
+Condition : isnotnull(cs_item_sk#53)
+
+(68) ReusedExchange [Reuses operator id: 46]
+Output [1]: [ss_item_sk#57]
+
+(69) CometBroadcastHashJoin
+Left output [4]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56]
+Right output [1]: [ss_item_sk#57]
+Arguments: [cs_item_sk#53], [ss_item_sk#57], LeftSemi, BuildRight
+
+(70) ReusedExchange [Reuses operator id: 52]
+Output [4]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61]
+
+(71) CometBroadcastHashJoin
+Left output [4]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56]
+Right output [4]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61]
+Arguments: [cs_item_sk#53], [i_item_sk#58], Inner, BuildRight
+
+(72) CometProject
+Input [8]: [cs_item_sk#53, cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61]
+Arguments: [cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_brand_id#59, i_class_id#60, i_category_id#61], [cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_brand_id#59, i_class_id#60, i_category_id#61]
+
+(73) ReusedExchange [Reuses operator id: 58]
+Output [1]: [d_date_sk#62]
+
+(74) CometBroadcastHashJoin
+Left output [6]: [cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_brand_id#59, i_class_id#60, i_category_id#61]
+Right output [1]: [d_date_sk#62]
+Arguments: [cs_sold_date_sk#56], [d_date_sk#62], Inner, BuildRight
+
+(75) CometProject
+Input [7]: [cs_quantity#54, cs_list_price#55, cs_sold_date_sk#56, i_brand_id#59, i_class_id#60, i_category_id#61, d_date_sk#62]
+Arguments: [cs_quantity#54, cs_list_price#55, i_brand_id#59, i_class_id#60, i_category_id#61], [cs_quantity#54, cs_list_price#55, i_brand_id#59, i_class_id#60, i_category_id#61]
+
+(76) CometHashAggregate
+Input [5]: [cs_quantity#54, cs_list_price#55, i_brand_id#59, i_class_id#60, i_category_id#61]
+Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
+Functions [2]: [partial_sum((cast(cs_quantity#54 as decimal(10,0)) * cs_list_price#55)), partial_count(1)]
+
+(77) CometExchange
+Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#63, isEmpty#64, count#65]
+Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(78) CometHashAggregate
+Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#63, isEmpty#64, count#65]
+Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
+Functions [2]: [sum((cast(cs_quantity#54 as decimal(10,0)) * cs_list_price#55)), count(1)]
+
+(79) CometFilter
+Input [5]: [i_brand_id#59, i_class_id#60, i_category_id#61, sales#66, number_sales#67]
+Condition : (isnotnull(sales#66) AND (cast(sales#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
+
+(80) CometProject
+Input [5]: [i_brand_id#59, i_class_id#60, i_category_id#61, sales#66, number_sales#67]
+Arguments: [sales#66, number_sales#67, channel#68, i_brand_id#59, i_class_id#60, i_category_id#61], [sales#66, number_sales#67, catalog AS channel#68, i_brand_id#59, i_class_id#60, i_category_id#61]
+
+(81) CometScan parquet spark_catalog.default.web_sales
+Output [4]: [ws_item_sk#69, ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#72)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(82) CometFilter
+Input [4]: [ws_item_sk#69, ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72]
+Condition : isnotnull(ws_item_sk#69)
+
+(83) ReusedExchange [Reuses operator id: 46]
+Output [1]: [ss_item_sk#73]
+
+(84) CometBroadcastHashJoin
+Left output [4]: [ws_item_sk#69, ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72]
+Right output [1]: [ss_item_sk#73]
+Arguments: [ws_item_sk#69], [ss_item_sk#73], LeftSemi, BuildRight
+
+(85) ReusedExchange [Reuses operator id: 52]
+Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77]
+
+(86) CometBroadcastHashJoin
+Left output [4]: [ws_item_sk#69, ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72]
+Right output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77]
+Arguments: [ws_item_sk#69], [i_item_sk#74], Inner, BuildRight
+
+(87) CometProject
+Input [8]: [ws_item_sk#69, ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77]
+Arguments: [ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77], [ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77]
+
+(88) ReusedExchange [Reuses operator id: 58]
+Output [1]: [d_date_sk#78]
+
+(89) CometBroadcastHashJoin
+Left output [6]: [ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77]
+Right output [1]: [d_date_sk#78]
+Arguments: [ws_sold_date_sk#72], [d_date_sk#78], Inner, BuildRight
+
+(90) CometProject
+Input [7]: [ws_quantity#70, ws_list_price#71, ws_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, d_date_sk#78]
+Arguments: [ws_quantity#70, ws_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77], [ws_quantity#70, ws_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77]
+
+(91) CometHashAggregate
+Input [5]: [ws_quantity#70, ws_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77]
+Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77]
+Functions [2]: [partial_sum((cast(ws_quantity#70 as decimal(10,0)) * ws_list_price#71)), partial_count(1)]
+
+(92) CometExchange
+Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#79, isEmpty#80, count#81]
+Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(93) CometHashAggregate
+Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#79, isEmpty#80, count#81]
+Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77]
+Functions [2]: [sum((cast(ws_quantity#70 as decimal(10,0)) * ws_list_price#71)), count(1)]
+
+(94) CometFilter
+Input [5]: [i_brand_id#75, i_class_id#76, i_category_id#77, sales#82, number_sales#83]
+Condition : (isnotnull(sales#82) AND (cast(sales#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
+
+(95) CometProject
+Input [5]: [i_brand_id#75, i_class_id#76, i_category_id#77, sales#82, number_sales#83]
+Arguments: [sales#82, number_sales#83, channel#84, i_brand_id#75, i_class_id#76, i_category_id#77], [sales#82, number_sales#83, web AS channel#84, i_brand_id#75, i_class_id#76, i_category_id#77]
+
+(96) CometUnion
+Child 0 Input [6]: [sales#45, number_sales#46, channel#49, i_brand_id#50, i_class_id#51, i_category_id#52]
+Child 1 Input [6]: [sales#66, number_sales#67, channel#68, i_brand_id#59, i_class_id#60, i_category_id#61]
+Child 2 Input [6]: [sales#82, number_sales#83, channel#84, i_brand_id#75, i_class_id#76, i_category_id#77]
+
+(97) CometExpand
+Input [6]: [sales#45, number_sales#46, channel#49, i_brand_id#50, i_class_id#51, i_category_id#52]
+Arguments: [[sales#45, number_sales#46, channel#49, i_brand_id#50, i_class_id#51, i_category_id#52, 0], [sales#45, number_sales#46, channel#49, i_brand_id#50, i_class_id#51, null, 1], [sales#45, number_sales#46, channel#49, i_brand_id#50, null, null, 3], [sales#45, number_sales#46, channel#49, null, null, null, 7], [sales#45, number_sales#46, null, null, null, null, 15]], [sales#45, number_sales#46, channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89]
+
+(98) CometHashAggregate
+Input [7]: [sales#45, number_sales#46, channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89]
+Keys [5]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89]
+Functions [2]: [partial_sum(sales#45), partial_sum(number_sales#46)]
+
+(99) CometExchange
+Input [8]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89, sum#90, isEmpty#91, sum#92]
+Arguments: hashpartitioning(channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5]
+
+(100) CometHashAggregate
+Input [8]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89, sum#90, isEmpty#91, sum#92]
+Keys [5]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, spark_grouping_id#89]
+Functions [2]: [sum(sales#45), sum(number_sales#46)]
+
+(101) CometTakeOrderedAndProject
+Input [6]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, sum(sales)#93, sum(number_sales)#94]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#85 ASC NULLS FIRST,i_brand_id#86 ASC NULLS FIRST,i_class_id#87 ASC NULLS FIRST,i_category_id#88 ASC NULLS FIRST], output=[channel#85,i_brand_id#86,i_class_id#87,i_category_id#88,sum(sales)#93,sum(number_sales)#94]), [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, sum(sales)#93, sum(number_sales)#94], 100, [channel#85 ASC NULLS FIRST, i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST], [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, sum(sales)#93, sum(number_sales)#94]
+
+(102) ColumnarToRow [codegen id : 1]
+Input [6]: [channel#85, i_brand_id#86, i_class_id#87, i_category_id#88, sum(sales)#93, sum(number_sales)#94]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#47, [id=#48]
+* ColumnarToRow (119)
++- CometHashAggregate (118)
+ +- CometExchange (117)
+ +- CometHashAggregate (116)
+ +- CometUnion (115)
+ :- CometProject (106)
+ : +- CometBroadcastHashJoin (105)
+ : :- CometScan parquet spark_catalog.default.store_sales (103)
+ : +- ReusedExchange (104)
+ :- CometProject (110)
+ : +- CometBroadcastHashJoin (109)
+ : :- CometScan parquet spark_catalog.default.catalog_sales (107)
+ : +- ReusedExchange (108)
+ +- CometProject (114)
+ +- CometBroadcastHashJoin (113)
+ :- CometScan parquet spark_catalog.default.web_sales (111)
+ +- ReusedExchange (112)
+
+
+(103) CometScan parquet spark_catalog.default.store_sales
+Output [3]: [ss_quantity#95, ss_list_price#96, ss_sold_date_sk#97]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#97)]
+ReadSchema: struct
+
+(104) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#98]
+
+(105) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#95, ss_list_price#96, ss_sold_date_sk#97]
+Right output [1]: [d_date_sk#98]
+Arguments: [ss_sold_date_sk#97], [d_date_sk#98], Inner, BuildRight
+
+(106) CometProject
+Input [4]: [ss_quantity#95, ss_list_price#96, ss_sold_date_sk#97, d_date_sk#98]
+Arguments: [quantity#99, list_price#100], [ss_quantity#95 AS quantity#99, ss_list_price#96 AS list_price#100]
+
+(107) CometScan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#103]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#103)]
+ReadSchema: struct
+
+(108) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#104]
+
+(109) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#103]
+Right output [1]: [d_date_sk#104]
+Arguments: [cs_sold_date_sk#103], [d_date_sk#104], Inner, BuildRight
+
+(110) CometProject
+Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#103, d_date_sk#104]
+Arguments: [quantity#105, list_price#106], [cs_quantity#101 AS quantity#105, cs_list_price#102 AS list_price#106]
+
+(111) CometScan parquet spark_catalog.default.web_sales
+Output [3]: [ws_quantity#107, ws_list_price#108, ws_sold_date_sk#109]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#109)]
+ReadSchema: struct
+
+(112) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#110]
+
+(113) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#107, ws_list_price#108, ws_sold_date_sk#109]
+Right output [1]: [d_date_sk#110]
+Arguments: [ws_sold_date_sk#109], [d_date_sk#110], Inner, BuildRight
+
+(114) CometProject
+Input [4]: [ws_quantity#107, ws_list_price#108, ws_sold_date_sk#109, d_date_sk#110]
+Arguments: [quantity#111, list_price#112], [ws_quantity#107 AS quantity#111, ws_list_price#108 AS list_price#112]
+
+(115) CometUnion
+Child 0 Input [2]: [quantity#99, list_price#100]
+Child 1 Input [2]: [quantity#105, list_price#106]
+Child 2 Input [2]: [quantity#111, list_price#112]
+
+(116) CometHashAggregate
+Input [2]: [quantity#99, list_price#100]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#99 as decimal(10,0)) * list_price#100))]
+
+(117) CometExchange
+Input [2]: [sum#113, count#114]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6]
+
+(118) CometHashAggregate
+Input [2]: [sum#113, count#114]
+Keys: []
+Functions [1]: [avg((cast(quantity#99 as decimal(10,0)) * list_price#100))]
+
+(119) ColumnarToRow [codegen id : 1]
+Input [1]: [average_sales#115]
+
+Subquery:2 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48]
+
+Subquery:3 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt
new file mode 100644
index 0000000000..f9ba3516f2
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt
@@ -0,0 +1,126 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)]
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),spark_grouping_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)]
+ CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales]
+ CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id]
+ CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ Subquery #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))]
+ CometExchange #13
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
+ CometProject [ss_quantity,ss_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometProject [cs_quantity,cs_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometProject [ws_quantity,ws_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedExchange [d_date_sk] #9
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #2
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price]
+ CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometBroadcastExchange [ss_item_sk] #3
+ CometProject [i_item_sk] [ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [brand_id,class_id,category_id] #4
+ CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometExchange [brand_id,class_id,category_id] #5
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #7
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [d_date_sk] #9
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ ReusedExchange [d_date_sk] #9
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ ReusedExchange [d_date_sk] #9
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ ReusedExchange [ss_item_sk] #3
+ CometBroadcastExchange [d_date_sk] #12
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ ReusedSubquery [average_sales] #1
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #14
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price]
+ CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedExchange [ss_item_sk] #3
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ ReusedExchange [d_date_sk] #12
+ CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ ReusedSubquery [average_sales] #1
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #15
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price]
+ CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedExchange [ss_item_sk] #3
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ ReusedExchange [d_date_sk] #12
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt
new file mode 100644
index 0000000000..5d8a967ff0
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt
@@ -0,0 +1,542 @@
+== Physical Plan ==
+* ColumnarToRow (77)
++- CometTakeOrderedAndProject (76)
+ +- CometBroadcastHashJoin (75)
+ :- CometFilter (56)
+ : +- CometHashAggregate (55)
+ : +- CometExchange (54)
+ : +- CometHashAggregate (53)
+ : +- CometProject (52)
+ : +- CometBroadcastHashJoin (51)
+ : :- CometProject (46)
+ : : +- CometBroadcastHashJoin (45)
+ : : :- CometBroadcastHashJoin (39)
+ : : : :- CometFilter (2)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (1)
+ : : : +- CometBroadcastExchange (38)
+ : : : +- CometProject (37)
+ : : : +- CometBroadcastHashJoin (36)
+ : : : :- CometFilter (4)
+ : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (3)
+ : : : +- CometBroadcastExchange (35)
+ : : : +- CometBroadcastHashJoin (34)
+ : : : :- CometHashAggregate (32)
+ : : : : +- CometExchange (31)
+ : : : : +- CometHashAggregate (30)
+ : : : : +- CometProject (29)
+ : : : : +- CometBroadcastHashJoin (28)
+ : : : : :- CometProject (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (6)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (5)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometBroadcastHashJoin (23)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (7)
+ : : : : : +- CometBroadcastExchange (22)
+ : : : : : +- CometProject (21)
+ : : : : : +- CometBroadcastHashJoin (20)
+ : : : : : :- CometProject (15)
+ : : : : : : +- CometBroadcastHashJoin (14)
+ : : : : : : :- CometFilter (10)
+ : : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`catalog_sales` (9)
+ : : : : : : +- CometBroadcastExchange (13)
+ : : : : : : +- CometFilter (12)
+ : : : : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (11)
+ : : : : : +- CometBroadcastExchange (19)
+ : : : : : +- CometProject (18)
+ : : : : : +- CometFilter (17)
+ : : : : : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (16)
+ : : : : +- ReusedExchange (27)
+ : : : +- ReusedExchange (33)
+ : : +- CometBroadcastExchange (44)
+ : : +- CometBroadcastHashJoin (43)
+ : : :- CometFilter (41)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`item` (40)
+ : : +- ReusedExchange (42)
+ : +- CometBroadcastExchange (50)
+ : +- CometProject (49)
+ : +- CometFilter (48)
+ : +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (47)
+ +- CometBroadcastExchange (74)
+ +- CometFilter (73)
+ +- CometHashAggregate (72)
+ +- CometExchange (71)
+ +- CometHashAggregate (70)
+ +- CometProject (69)
+ +- CometBroadcastHashJoin (68)
+ :- CometProject (63)
+ : +- CometBroadcastHashJoin (62)
+ : :- CometBroadcastHashJoin (60)
+ : : :- CometFilter (58)
+ : : : +- CometNativeScan: `spark_catalog`.`default`.`store_sales` (57)
+ : : +- ReusedExchange (59)
+ : +- ReusedExchange (61)
+ +- CometBroadcastExchange (67)
+ +- CometProject (66)
+ +- CometFilter (65)
+ +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (64)
+
+
+(1) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Arguments: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Arguments: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+
+(4) CometFilter
+Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8))
+
+(5) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Arguments: [ss_item_sk#9, ss_sold_date_sk#10]
+
+(6) CometFilter
+Input [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Condition : isnotnull(ss_item_sk#9)
+
+(7) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(8) CometFilter
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14))
+
+(9) CometNativeScan: `spark_catalog`.`default`.`catalog_sales`
+Output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Arguments: [cs_item_sk#15, cs_sold_date_sk#16]
+
+(10) CometFilter
+Input [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Condition : isnotnull(cs_item_sk#15)
+
+(11) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(12) CometFilter
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Condition : isnotnull(i_item_sk#17)
+
+(13) CometBroadcastExchange
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(14) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Right output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_item_sk#15], [i_item_sk#17], Inner, BuildRight
+
+(15) CometProject
+Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20], [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(16) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21, d_year#22]
+
+(17) CometFilter
+Input [2]: [d_date_sk#21, d_year#22]
+Condition : (((isnotnull(d_year#22) AND (d_year#22 >= 1999)) AND (d_year#22 <= 2001)) AND isnotnull(d_date_sk#21))
+
+(18) CometProject
+Input [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(19) CometBroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: [d_date_sk#21]
+
+(20) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+Right output [1]: [d_date_sk#21]
+Arguments: [cs_sold_date_sk#16], [d_date_sk#21], Inner, BuildRight
+
+(21) CometProject
+Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#21]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20], [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(22) CometBroadcastExchange
+Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(23) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)], [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)], LeftSemi, BuildRight
+
+(24) CometBroadcastExchange
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(25) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Right output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_item_sk#9], [i_item_sk#11], Inner, BuildRight
+
+(26) CometProject
+Input [6]: [ss_item_sk#9, ss_sold_date_sk#10, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14], [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(27) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#23]
+
+(28) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [1]: [d_date_sk#23]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#23], Inner, BuildRight
+
+(29) CometProject
+Input [5]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#23]
+Arguments: [brand_id#24, class_id#25, category_id#26], [i_brand_id#12 AS brand_id#24, i_class_id#13 AS class_id#25, i_category_id#14 AS category_id#26]
+
+(30) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(31) CometExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: hashpartitioning(brand_id#24, class_id#25, category_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(33) ReusedExchange [Reuses operator id: 22]
+Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29]
+
+(34) CometBroadcastHashJoin
+Left output [3]: [brand_id#24, class_id#25, category_id#26]
+Right output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29]
+Arguments: [coalesce(brand_id#24, 0), isnull(brand_id#24), coalesce(class_id#25, 0), isnull(class_id#25), coalesce(category_id#26, 0), isnull(category_id#26)], [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)], LeftSemi, BuildRight
+
+(35) CometBroadcastExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [brand_id#24, class_id#25, category_id#26]
+
+(36) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Right output [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [i_brand_id#6, i_class_id#7, i_category_id#8], [brand_id#24, class_id#25, category_id#26], Inner, BuildRight
+
+(37) CometProject
+Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#24, class_id#25, category_id#26]
+Arguments: [ss_item_sk#30], [i_item_sk#5 AS ss_item_sk#30]
+
+(38) CometBroadcastExchange
+Input [1]: [ss_item_sk#30]
+Arguments: [ss_item_sk#30]
+
+(39) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [1]: [ss_item_sk#30]
+Arguments: [ss_item_sk#1], [ss_item_sk#30], LeftSemi, BuildRight
+
+(40) CometNativeScan: `spark_catalog`.`default`.`item`
+Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(41) CometFilter
+Input [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Condition : (((isnotnull(i_item_sk#31) AND isnotnull(i_brand_id#32)) AND isnotnull(i_class_id#33)) AND isnotnull(i_category_id#34))
+
+(42) ReusedExchange [Reuses operator id: 38]
+Output [1]: [ss_item_sk#30]
+
+(43) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Right output [1]: [ss_item_sk#30]
+Arguments: [i_item_sk#31], [ss_item_sk#30], LeftSemi, BuildRight
+
+(44) CometBroadcastExchange
+Input [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(45) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight
+
+(46) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
+Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(47) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#35, d_week_seq#36]
+Arguments: [d_date_sk#35, d_week_seq#36]
+
+(48) CometFilter
+Input [2]: [d_date_sk#35, d_week_seq#36]
+Condition : ((isnotnull(d_week_seq#36) AND (d_week_seq#36 = Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#35))
+
+(49) CometProject
+Input [2]: [d_date_sk#35, d_week_seq#36]
+Arguments: [d_date_sk#35], [d_date_sk#35]
+
+(50) CometBroadcastExchange
+Input [1]: [d_date_sk#35]
+Arguments: [d_date_sk#35]
+
+(51) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34]
+Right output [1]: [d_date_sk#35]
+Arguments: [ss_sold_date_sk#4], [d_date_sk#35], Inner, BuildRight
+
+(52) CometProject
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#32, i_class_id#33, i_category_id#34, d_date_sk#35]
+Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34], [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34]
+
+(53) CometHashAggregate
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#32, i_class_id#33, i_category_id#34]
+Keys [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+
+(54) CometExchange
+Input [6]: [i_brand_id#32, i_class_id#33, i_category_id#34, sum#39, isEmpty#40, count#41]
+Arguments: hashpartitioning(i_brand_id#32, i_class_id#33, i_category_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(55) CometHashAggregate
+Input [6]: [i_brand_id#32, i_class_id#33, i_category_id#34, sum#39, isEmpty#40, count#41]
+Keys [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+
+(56) CometFilter
+Input [6]: [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44]
+Condition : (isnotnull(sales#43) AND (cast(sales#43 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
+
+(57) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [4]: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50]
+Arguments: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50]
+
+(58) CometFilter
+Input [4]: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50]
+Condition : isnotnull(ss_item_sk#47)
+
+(59) ReusedExchange [Reuses operator id: 38]
+Output [1]: [ss_item_sk#51]
+
+(60) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50]
+Right output [1]: [ss_item_sk#51]
+Arguments: [ss_item_sk#47], [ss_item_sk#51], LeftSemi, BuildRight
+
+(61) ReusedExchange [Reuses operator id: 44]
+Output [4]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55]
+
+(62) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50]
+Right output [4]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55]
+Arguments: [ss_item_sk#47], [i_item_sk#52], Inner, BuildRight
+
+(63) CometProject
+Input [8]: [ss_item_sk#47, ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55]
+Arguments: [ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50, i_brand_id#53, i_class_id#54, i_category_id#55], [ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50, i_brand_id#53, i_class_id#54, i_category_id#55]
+
+(64) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [2]: [d_date_sk#56, d_week_seq#57]
+Arguments: [d_date_sk#56, d_week_seq#57]
+
+(65) CometFilter
+Input [2]: [d_date_sk#56, d_week_seq#57]
+Condition : ((isnotnull(d_week_seq#57) AND (d_week_seq#57 = Subquery scalar-subquery#58, [id=#59])) AND isnotnull(d_date_sk#56))
+
+(66) CometProject
+Input [2]: [d_date_sk#56, d_week_seq#57]
+Arguments: [d_date_sk#56], [d_date_sk#56]
+
+(67) CometBroadcastExchange
+Input [1]: [d_date_sk#56]
+Arguments: [d_date_sk#56]
+
+(68) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50, i_brand_id#53, i_class_id#54, i_category_id#55]
+Right output [1]: [d_date_sk#56]
+Arguments: [ss_sold_date_sk#50], [d_date_sk#56], Inner, BuildRight
+
+(69) CometProject
+Input [7]: [ss_quantity#48, ss_list_price#49, ss_sold_date_sk#50, i_brand_id#53, i_class_id#54, i_category_id#55, d_date_sk#56]
+Arguments: [ss_quantity#48, ss_list_price#49, i_brand_id#53, i_class_id#54, i_category_id#55], [ss_quantity#48, ss_list_price#49, i_brand_id#53, i_class_id#54, i_category_id#55]
+
+(70) CometHashAggregate
+Input [5]: [ss_quantity#48, ss_list_price#49, i_brand_id#53, i_class_id#54, i_category_id#55]
+Keys [3]: [i_brand_id#53, i_class_id#54, i_category_id#55]
+Functions [2]: [partial_sum((cast(ss_quantity#48 as decimal(10,0)) * ss_list_price#49)), partial_count(1)]
+
+(71) CometExchange
+Input [6]: [i_brand_id#53, i_class_id#54, i_category_id#55, sum#60, isEmpty#61, count#62]
+Arguments: hashpartitioning(i_brand_id#53, i_class_id#54, i_category_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(72) CometHashAggregate
+Input [6]: [i_brand_id#53, i_class_id#54, i_category_id#55, sum#60, isEmpty#61, count#62]
+Keys [3]: [i_brand_id#53, i_class_id#54, i_category_id#55]
+Functions [2]: [sum((cast(ss_quantity#48 as decimal(10,0)) * ss_list_price#49)), count(1)]
+
+(73) CometFilter
+Input [6]: [channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+Condition : (isnotnull(sales#64) AND (cast(sales#64 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
+
+(74) CometBroadcastExchange
+Input [6]: [channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+Arguments: [channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+
+(75) CometBroadcastHashJoin
+Left output [6]: [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44]
+Right output [6]: [channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+Arguments: [i_brand_id#32, i_class_id#33, i_category_id#34], [i_brand_id#53, i_class_id#54, i_category_id#55], Inner, BuildRight
+
+(76) CometTakeOrderedAndProject
+Input [12]: [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44, channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#32 ASC NULLS FIRST,i_class_id#33 ASC NULLS FIRST,i_category_id#34 ASC NULLS FIRST], output=[channel#42,i_brand_id#32,i_class_id#33,i_category_id#34,sales#43,number_sales#44,channel#63,i_brand_id#53,i_class_id#54,i_category_id#55,sales#64,number_sales#65]), [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44, channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65], 100, [i_brand_id#32 ASC NULLS FIRST, i_class_id#33 ASC NULLS FIRST, i_category_id#34 ASC NULLS FIRST], [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44, channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+
+(77) ColumnarToRow [codegen id : 1]
+Input [12]: [channel#42, i_brand_id#32, i_class_id#33, i_category_id#34, sales#43, number_sales#44, channel#63, i_brand_id#53, i_class_id#54, i_category_id#55, sales#64, number_sales#65]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#45, [id=#46]
+* ColumnarToRow (94)
++- CometHashAggregate (93)
+ +- CometExchange (92)
+ +- CometHashAggregate (91)
+ +- CometUnion (90)
+ :- CometProject (81)
+ : +- CometBroadcastHashJoin (80)
+ : :- CometNativeScan: `spark_catalog`.`default`.`store_sales` (78)
+ : +- ReusedExchange (79)
+ :- CometProject (85)
+ : +- CometBroadcastHashJoin (84)
+ : :- CometNativeScan: `spark_catalog`.`default`.`catalog_sales` (82)
+ : +- ReusedExchange (83)
+ +- CometProject (89)
+ +- CometBroadcastHashJoin (88)
+ :- CometNativeScan: `spark_catalog`.`default`.`web_sales` (86)
+ +- ReusedExchange (87)
+
+
+(78) CometNativeScan: `spark_catalog`.`default`.`store_sales`
+Output [3]: [ss_quantity#66, ss_list_price#67, ss_sold_date_sk#68]
+Arguments: [ss_quantity#66, ss_list_price#67, ss_sold_date_sk#68]
+
+(79) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#69]
+
+(80) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#66, ss_list_price#67, ss_sold_date_sk#68]
+Right output [1]: [d_date_sk#69]
+Arguments: [ss_sold_date_sk#68], [d_date_sk#69], Inner, BuildRight
+
+(81) CometProject
+Input [4]: [ss_quantity#66, ss_list_price#67, ss_sold_date_sk#68, d_date_sk#69]
+Arguments: [quantity#70, list_price#71], [ss_quantity#66 AS quantity#70, ss_list_price#67 AS list_price#71]
+
+(82) CometNativeScan: `spark_catalog`.`default`.`catalog_sales`
+Output [3]: [cs_quantity#72, cs_list_price#73, cs_sold_date_sk#74]
+Arguments: [cs_quantity#72, cs_list_price#73, cs_sold_date_sk#74]
+
+(83) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#75]
+
+(84) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#72, cs_list_price#73, cs_sold_date_sk#74]
+Right output [1]: [d_date_sk#75]
+Arguments: [cs_sold_date_sk#74], [d_date_sk#75], Inner, BuildRight
+
+(85) CometProject
+Input [4]: [cs_quantity#72, cs_list_price#73, cs_sold_date_sk#74, d_date_sk#75]
+Arguments: [quantity#76, list_price#77], [cs_quantity#72 AS quantity#76, cs_list_price#73 AS list_price#77]
+
+(86) CometNativeScan: `spark_catalog`.`default`.`web_sales`
+Output [3]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
+Arguments: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
+
+(87) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#81]
+
+(88) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
+Right output [1]: [d_date_sk#81]
+Arguments: [ws_sold_date_sk#80], [d_date_sk#81], Inner, BuildRight
+
+(89) CometProject
+Input [4]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, d_date_sk#81]
+Arguments: [quantity#82, list_price#83], [ws_quantity#78 AS quantity#82, ws_list_price#79 AS list_price#83]
+
+(90) CometUnion
+Child 0 Input [2]: [quantity#70, list_price#71]
+Child 1 Input [2]: [quantity#76, list_price#77]
+Child 2 Input [2]: [quantity#82, list_price#83]
+
+(91) CometHashAggregate
+Input [2]: [quantity#70, list_price#71]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#70 as decimal(10,0)) * list_price#71))]
+
+(92) CometExchange
+Input [2]: [sum#84, count#85]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(93) CometHashAggregate
+Input [2]: [sum#84, count#85]
+Keys: []
+Functions [1]: [avg((cast(quantity#70 as decimal(10,0)) * list_price#71))]
+
+(94) ColumnarToRow [codegen id : 1]
+Input [1]: [average_sales#86]
+
+Subquery:2 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#37, [id=#38]
+* ColumnarToRow (98)
++- CometProject (97)
+ +- CometFilter (96)
+ +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (95)
+
+
+(95) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [4]: [d_week_seq#87, d_year#88, d_moy#89, d_dom#90]
+Arguments: [d_week_seq#87, d_year#88, d_moy#89, d_dom#90]
+
+(96) CometFilter
+Input [4]: [d_week_seq#87, d_year#88, d_moy#89, d_dom#90]
+Condition : (((((isnotnull(d_year#88) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#88 = 2000)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11))
+
+(97) CometProject
+Input [4]: [d_week_seq#87, d_year#88, d_moy#89, d_dom#90]
+Arguments: [d_week_seq#87], [d_week_seq#87]
+
+(98) ColumnarToRow [codegen id : 1]
+Input [1]: [d_week_seq#87]
+
+Subquery:3 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
+
+Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#58, [id=#59]
+* ColumnarToRow (102)
++- CometProject (101)
+ +- CometFilter (100)
+ +- CometNativeScan: `spark_catalog`.`default`.`date_dim` (99)
+
+
+(99) CometNativeScan: `spark_catalog`.`default`.`date_dim`
+Output [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Arguments: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+
+(100) CometFilter
+Input [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#93)) AND isnotnull(d_dom#94)) AND (d_year#92 = 1999)) AND (d_moy#93 = 12)) AND (d_dom#94 = 11))
+
+(101) CometProject
+Input [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Arguments: [d_week_seq#91], [d_week_seq#91]
+
+(102) ColumnarToRow [codegen id : 1]
+Input [1]: [d_week_seq#91]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt
new file mode 100644
index 0000000000..d5143df8c8
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt
@@ -0,0 +1,114 @@
+WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ Subquery #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))]
+ CometExchange #11
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
+ CometProject [ss_quantity,ss_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedExchange [d_date_sk] #8
+ CometProject [cs_quantity,cs_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`catalog_sales` [cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedExchange [d_date_sk] #8
+ CometProject [ws_quantity,ws_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`web_sales` [ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedExchange [d_date_sk] #8
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #1
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price]
+ CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometBroadcastExchange [ss_item_sk] #2
+ CometProject [i_item_sk] [ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [brand_id,class_id,category_id] #3
+ CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometExchange [brand_id,class_id,category_id] #4
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_item_sk,ss_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #5
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #6
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`catalog_sales` [cs_item_sk,cs_sold_date_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [d_date_sk] #8
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_year]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_year]
+ ReusedExchange [d_date_sk] #8
+ ReusedExchange [i_brand_id,i_class_id,i_category_id] #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometNativeScan: `spark_catalog`.`default`.`item` [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ ReusedExchange [ss_item_sk] #2
+ CometBroadcastExchange [d_date_sk] #10
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
+ Subquery #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_week_seq,d_year,d_moy,d_dom]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_week_seq]
+ CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #12
+ CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
+ ReusedSubquery [average_sales] #2
+ CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)]
+ CometExchange [i_brand_id,i_class_id,i_category_id] #13
+ CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price]
+ CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk]
+ CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ CometNativeScan: `spark_catalog`.`default`.`store_sales` [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedExchange [ss_item_sk] #2
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ CometBroadcastExchange [d_date_sk] #14
+ CometProject [d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
+ Subquery #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_week_seq,d_year,d_moy,d_dom]
+ CometNativeScan: `spark_catalog`.`default`.`date_dim` [d_date_sk,d_week_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt
new file mode 100644
index 0000000000..22c4967421
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt
@@ -0,0 +1,639 @@
+== Physical Plan ==
+* ColumnarToRow (85)
++- CometTakeOrderedAndProject (84)
+ +- CometBroadcastHashJoin (83)
+ :- CometFilter (64)
+ : +- CometHashAggregate (63)
+ : +- CometExchange (62)
+ : +- CometHashAggregate (61)
+ : +- CometProject (60)
+ : +- CometBroadcastHashJoin (59)
+ : :- CometProject (54)
+ : : +- CometBroadcastHashJoin (53)
+ : : :- CometBroadcastHashJoin (47)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- CometBroadcastExchange (46)
+ : : : +- CometProject (45)
+ : : : +- CometBroadcastHashJoin (44)
+ : : : :- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.item (3)
+ : : : +- CometBroadcastExchange (43)
+ : : : +- CometBroadcastHashJoin (42)
+ : : : :- CometHashAggregate (32)
+ : : : : +- CometExchange (31)
+ : : : : +- CometHashAggregate (30)
+ : : : : +- CometProject (29)
+ : : : : +- CometBroadcastHashJoin (28)
+ : : : : :- CometProject (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (6)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (5)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometBroadcastHashJoin (23)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (7)
+ : : : : : +- CometBroadcastExchange (22)
+ : : : : : +- CometProject (21)
+ : : : : : +- CometBroadcastHashJoin (20)
+ : : : : : :- CometProject (15)
+ : : : : : : +- CometBroadcastHashJoin (14)
+ : : : : : : :- CometFilter (10)
+ : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (9)
+ : : : : : : +- CometBroadcastExchange (13)
+ : : : : : : +- CometFilter (12)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (11)
+ : : : : : +- CometBroadcastExchange (19)
+ : : : : : +- CometProject (18)
+ : : : : : +- CometFilter (17)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (16)
+ : : : : +- ReusedExchange (27)
+ : : : +- CometBroadcastExchange (41)
+ : : : +- CometProject (40)
+ : : : +- CometBroadcastHashJoin (39)
+ : : : :- CometProject (37)
+ : : : : +- CometBroadcastHashJoin (36)
+ : : : : :- CometFilter (34)
+ : : : : : +- CometScan parquet spark_catalog.default.web_sales (33)
+ : : : : +- ReusedExchange (35)
+ : : : +- ReusedExchange (38)
+ : : +- CometBroadcastExchange (52)
+ : : +- CometBroadcastHashJoin (51)
+ : : :- CometFilter (49)
+ : : : +- CometScan parquet spark_catalog.default.item (48)
+ : : +- ReusedExchange (50)
+ : +- CometBroadcastExchange (58)
+ : +- CometProject (57)
+ : +- CometFilter (56)
+ : +- CometScan parquet spark_catalog.default.date_dim (55)
+ +- CometBroadcastExchange (82)
+ +- CometFilter (81)
+ +- CometHashAggregate (80)
+ +- CometExchange (79)
+ +- CometHashAggregate (78)
+ +- CometProject (77)
+ +- CometBroadcastHashJoin (76)
+ :- CometProject (71)
+ : +- CometBroadcastHashJoin (70)
+ : :- CometBroadcastHashJoin (68)
+ : : :- CometFilter (66)
+ : : : +- CometScan parquet spark_catalog.default.store_sales (65)
+ : : +- ReusedExchange (67)
+ : +- ReusedExchange (69)
+ +- CometBroadcastExchange (75)
+ +- CometProject (74)
+ +- CometFilter (73)
+ +- CometScan parquet spark_catalog.default.date_dim (72)
+
+
+(1) CometScan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#4)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8))
+
+(5) CometScan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#10)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(6) CometFilter
+Input [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Condition : isnotnull(ss_item_sk#9)
+
+(7) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14))
+
+(9) CometScan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#16)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Condition : isnotnull(cs_item_sk#15)
+
+(11) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(12) CometFilter
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Condition : isnotnull(i_item_sk#17)
+
+(13) CometBroadcastExchange
+Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(14) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#15, cs_sold_date_sk#16]
+Right output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_item_sk#15], [i_item_sk#17], Inner, BuildRight
+
+(15) CometProject
+Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20], [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(16) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#21, d_year#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(17) CometFilter
+Input [2]: [d_date_sk#21, d_year#22]
+Condition : (((isnotnull(d_year#22) AND (d_year#22 >= 1999)) AND (d_year#22 <= 2001)) AND isnotnull(d_date_sk#21))
+
+(18) CometProject
+Input [2]: [d_date_sk#21, d_year#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(19) CometBroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: [d_date_sk#21]
+
+(20) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20]
+Right output [1]: [d_date_sk#21]
+Arguments: [cs_sold_date_sk#16], [d_date_sk#21], Inner, BuildRight
+
+(21) CometProject
+Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#21]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20], [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(22) CometBroadcastExchange
+Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [i_brand_id#18, i_class_id#19, i_category_id#20]
+
+(23) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20]
+Arguments: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)], [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)], LeftSemi, BuildRight
+
+(24) CometBroadcastExchange
+Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(25) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#9, ss_sold_date_sk#10]
+Right output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_item_sk#9], [i_item_sk#11], Inner, BuildRight
+
+(26) CometProject
+Input [6]: [ss_item_sk#9, ss_sold_date_sk#10, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14]
+Arguments: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14], [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+
+(27) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#23]
+
+(28) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14]
+Right output [1]: [d_date_sk#23]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#23], Inner, BuildRight
+
+(29) CometProject
+Input [5]: [ss_sold_date_sk#10, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#23]
+Arguments: [brand_id#24, class_id#25, category_id#26], [i_brand_id#12 AS brand_id#24, i_class_id#13 AS class_id#25, i_category_id#14 AS category_id#26]
+
+(30) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(31) CometExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: hashpartitioning(brand_id#24, class_id#25, category_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Keys [3]: [brand_id#24, class_id#25, category_id#26]
+Functions: []
+
+(33) CometScan parquet spark_catalog.default.web_sales
+Output [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#28)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Condition : isnotnull(ws_item_sk#27)
+
+(35) ReusedExchange [Reuses operator id: 13]
+Output [4]: [i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(36) CometBroadcastHashJoin
+Left output [2]: [ws_item_sk#27, ws_sold_date_sk#28]
+Right output [4]: [i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [ws_item_sk#27], [i_item_sk#29], Inner, BuildRight
+
+(37) CometProject
+Input [6]: [ws_item_sk#27, ws_sold_date_sk#28, i_item_sk#29, i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32], [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(38) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#33]
+
+(39) CometBroadcastHashJoin
+Left output [4]: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32]
+Right output [1]: [d_date_sk#33]
+Arguments: [ws_sold_date_sk#28], [d_date_sk#33], Inner, BuildRight
+
+(40) CometProject
+Input [5]: [ws_sold_date_sk#28, i_brand_id#30, i_class_id#31, i_category_id#32, d_date_sk#33]
+Arguments: [i_brand_id#30, i_class_id#31, i_category_id#32], [i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(41) CometBroadcastExchange
+Input [3]: [i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [i_brand_id#30, i_class_id#31, i_category_id#32]
+
+(42) CometBroadcastHashJoin
+Left output [3]: [brand_id#24, class_id#25, category_id#26]
+Right output [3]: [i_brand_id#30, i_class_id#31, i_category_id#32]
+Arguments: [coalesce(brand_id#24, 0), isnull(brand_id#24), coalesce(class_id#25, 0), isnull(class_id#25), coalesce(category_id#26, 0), isnull(category_id#26)], [coalesce(i_brand_id#30, 0), isnull(i_brand_id#30), coalesce(i_class_id#31, 0), isnull(i_class_id#31), coalesce(i_category_id#32, 0), isnull(i_category_id#32)], LeftSemi, BuildRight
+
+(43) CometBroadcastExchange
+Input [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [brand_id#24, class_id#25, category_id#26]
+
+(44) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8]
+Right output [3]: [brand_id#24, class_id#25, category_id#26]
+Arguments: [i_brand_id#6, i_class_id#7, i_category_id#8], [brand_id#24, class_id#25, category_id#26], Inner, BuildRight
+
+(45) CometProject
+Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#24, class_id#25, category_id#26]
+Arguments: [ss_item_sk#34], [i_item_sk#5 AS ss_item_sk#34]
+
+(46) CometBroadcastExchange
+Input [1]: [ss_item_sk#34]
+Arguments: [ss_item_sk#34]
+
+(47) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [1]: [ss_item_sk#34]
+Arguments: [ss_item_sk#1], [ss_item_sk#34], LeftSemi, BuildRight
+
+(48) CometScan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
+ReadSchema: struct
+
+(49) CometFilter
+Input [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Condition : (((isnotnull(i_item_sk#35) AND isnotnull(i_brand_id#36)) AND isnotnull(i_class_id#37)) AND isnotnull(i_category_id#38))
+
+(50) ReusedExchange [Reuses operator id: 46]
+Output [1]: [ss_item_sk#34]
+
+(51) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Right output [1]: [ss_item_sk#34]
+Arguments: [i_item_sk#35], [ss_item_sk#34], LeftSemi, BuildRight
+
+(52) CometBroadcastExchange
+Input [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(53) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Right output [4]: [i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [ss_item_sk#1], [i_item_sk#35], Inner, BuildRight
+
+(54) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#35, i_brand_id#36, i_class_id#37, i_category_id#38]
+Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(55) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#39, d_week_seq#40]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(56) CometFilter
+Input [2]: [d_date_sk#39, d_week_seq#40]
+Condition : ((isnotnull(d_week_seq#40) AND (d_week_seq#40 = Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#39))
+
+(57) CometProject
+Input [2]: [d_date_sk#39, d_week_seq#40]
+Arguments: [d_date_sk#39], [d_date_sk#39]
+
+(58) CometBroadcastExchange
+Input [1]: [d_date_sk#39]
+Arguments: [d_date_sk#39]
+
+(59) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38]
+Right output [1]: [d_date_sk#39]
+Arguments: [ss_sold_date_sk#4], [d_date_sk#39], Inner, BuildRight
+
+(60) CometProject
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#36, i_class_id#37, i_category_id#38, d_date_sk#39]
+Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38], [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38]
+
+(61) CometHashAggregate
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#36, i_class_id#37, i_category_id#38]
+Keys [3]: [i_brand_id#36, i_class_id#37, i_category_id#38]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+
+(62) CometExchange
+Input [6]: [i_brand_id#36, i_class_id#37, i_category_id#38, sum#43, isEmpty#44, count#45]
+Arguments: hashpartitioning(i_brand_id#36, i_class_id#37, i_category_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2]
+
+(63) CometHashAggregate
+Input [6]: [i_brand_id#36, i_class_id#37, i_category_id#38, sum#43, isEmpty#44, count#45]
+Keys [3]: [i_brand_id#36, i_class_id#37, i_category_id#38]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+
+(64) CometFilter
+Input [6]: [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48]
+Condition : (isnotnull(sales#47) AND (cast(sales#47 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6))))
+
+(65) CometScan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#51, ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#54)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(66) CometFilter
+Input [4]: [ss_item_sk#51, ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54]
+Condition : isnotnull(ss_item_sk#51)
+
+(67) ReusedExchange [Reuses operator id: 46]
+Output [1]: [ss_item_sk#55]
+
+(68) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#51, ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54]
+Right output [1]: [ss_item_sk#55]
+Arguments: [ss_item_sk#51], [ss_item_sk#55], LeftSemi, BuildRight
+
+(69) ReusedExchange [Reuses operator id: 52]
+Output [4]: [i_item_sk#56, i_brand_id#57, i_class_id#58, i_category_id#59]
+
+(70) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#51, ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54]
+Right output [4]: [i_item_sk#56, i_brand_id#57, i_class_id#58, i_category_id#59]
+Arguments: [ss_item_sk#51], [i_item_sk#56], Inner, BuildRight
+
+(71) CometProject
+Input [8]: [ss_item_sk#51, ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54, i_item_sk#56, i_brand_id#57, i_class_id#58, i_category_id#59]
+Arguments: [ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54, i_brand_id#57, i_class_id#58, i_category_id#59], [ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54, i_brand_id#57, i_class_id#58, i_category_id#59]
+
+(72) CometScan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#60, d_week_seq#61]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(73) CometFilter
+Input [2]: [d_date_sk#60, d_week_seq#61]
+Condition : ((isnotnull(d_week_seq#61) AND (d_week_seq#61 = Subquery scalar-subquery#62, [id=#63])) AND isnotnull(d_date_sk#60))
+
+(74) CometProject
+Input [2]: [d_date_sk#60, d_week_seq#61]
+Arguments: [d_date_sk#60], [d_date_sk#60]
+
+(75) CometBroadcastExchange
+Input [1]: [d_date_sk#60]
+Arguments: [d_date_sk#60]
+
+(76) CometBroadcastHashJoin
+Left output [6]: [ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54, i_brand_id#57, i_class_id#58, i_category_id#59]
+Right output [1]: [d_date_sk#60]
+Arguments: [ss_sold_date_sk#54], [d_date_sk#60], Inner, BuildRight
+
+(77) CometProject
+Input [7]: [ss_quantity#52, ss_list_price#53, ss_sold_date_sk#54, i_brand_id#57, i_class_id#58, i_category_id#59, d_date_sk#60]
+Arguments: [ss_quantity#52, ss_list_price#53, i_brand_id#57, i_class_id#58, i_category_id#59], [ss_quantity#52, ss_list_price#53, i_brand_id#57, i_class_id#58, i_category_id#59]
+
+(78) CometHashAggregate
+Input [5]: [ss_quantity#52, ss_list_price#53, i_brand_id#57, i_class_id#58, i_category_id#59]
+Keys [3]: [i_brand_id#57, i_class_id#58, i_category_id#59]
+Functions [2]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_list_price#53)), partial_count(1)]
+
+(79) CometExchange
+Input [6]: [i_brand_id#57, i_class_id#58, i_category_id#59, sum#64, isEmpty#65, count#66]
+Arguments: hashpartitioning(i_brand_id#57, i_class_id#58, i_category_id#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3]
+
+(80) CometHashAggregate
+Input [6]: [i_brand_id#57, i_class_id#58, i_category_id#59, sum#64, isEmpty#65, count#66]
+Keys [3]: [i_brand_id#57, i_class_id#58, i_category_id#59]
+Functions [2]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_list_price#53)), count(1)]
+
+(81) CometFilter
+Input [6]: [channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6))))
+
+(82) CometBroadcastExchange
+Input [6]: [channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+Arguments: [channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+
+(83) CometBroadcastHashJoin
+Left output [6]: [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48]
+Right output [6]: [channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+Arguments: [i_brand_id#36, i_class_id#37, i_category_id#38], [i_brand_id#57, i_class_id#58, i_category_id#59], Inner, BuildRight
+
+(84) CometTakeOrderedAndProject
+Input [12]: [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48, channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#36 ASC NULLS FIRST,i_class_id#37 ASC NULLS FIRST,i_category_id#38 ASC NULLS FIRST], output=[channel#46,i_brand_id#36,i_class_id#37,i_category_id#38,sales#47,number_sales#48,channel#67,i_brand_id#57,i_class_id#58,i_category_id#59,sales#68,number_sales#69]), [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48, channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69], 100, [i_brand_id#36 ASC NULLS FIRST, i_class_id#37 ASC NULLS FIRST, i_category_id#38 ASC NULLS FIRST], [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48, channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+
+(85) ColumnarToRow [codegen id : 1]
+Input [12]: [channel#46, i_brand_id#36, i_class_id#37, i_category_id#38, sales#47, number_sales#48, channel#67, i_brand_id#57, i_class_id#58, i_category_id#59, sales#68, number_sales#69]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#49, [id=#50]
+* ColumnarToRow (102)
++- CometHashAggregate (101)
+ +- CometExchange (100)
+ +- CometHashAggregate (99)
+ +- CometUnion (98)
+ :- CometProject (89)
+ : +- CometBroadcastHashJoin (88)
+ : :- CometScan parquet spark_catalog.default.store_sales (86)
+ : +- ReusedExchange (87)
+ :- CometProject (93)
+ : +- CometBroadcastHashJoin (92)
+ : :- CometScan parquet spark_catalog.default.catalog_sales (90)
+ : +- ReusedExchange (91)
+ +- CometProject (97)
+ +- CometBroadcastHashJoin (96)
+ :- CometScan parquet spark_catalog.default.web_sales (94)
+ +- ReusedExchange (95)
+
+
+(86) CometScan parquet spark_catalog.default.store_sales
+Output [3]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#72)]
+ReadSchema: struct
+
+(87) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#73]
+
+(88) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72]
+Right output [1]: [d_date_sk#73]
+Arguments: [ss_sold_date_sk#72], [d_date_sk#73], Inner, BuildRight
+
+(89) CometProject
+Input [4]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, d_date_sk#73]
+Arguments: [quantity#74, list_price#75], [ss_quantity#70 AS quantity#74, ss_list_price#71 AS list_price#75]
+
+(90) CometScan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_quantity#76, cs_list_price#77, cs_sold_date_sk#78]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#78)]
+ReadSchema: struct
+
+(91) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#79]
+
+(92) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#76, cs_list_price#77, cs_sold_date_sk#78]
+Right output [1]: [d_date_sk#79]
+Arguments: [cs_sold_date_sk#78], [d_date_sk#79], Inner, BuildRight
+
+(93) CometProject
+Input [4]: [cs_quantity#76, cs_list_price#77, cs_sold_date_sk#78, d_date_sk#79]
+Arguments: [quantity#80, list_price#81], [cs_quantity#76 AS quantity#80, cs_list_price#77 AS list_price#81]
+
+(94) CometScan parquet spark_catalog.default.web_sales
+Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#84)]
+ReadSchema: struct
+
+(95) ReusedExchange [Reuses operator id: 19]
+Output [1]: [d_date_sk#85]
+
+(96) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84]
+Right output [1]: [d_date_sk#85]
+Arguments: [ws_sold_date_sk#84], [d_date_sk#85], Inner, BuildRight
+
+(97) CometProject
+Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, d_date_sk#85]
+Arguments: [quantity#86, list_price#87], [ws_quantity#82 AS quantity#86, ws_list_price#83 AS list_price#87]
+
+(98) CometUnion
+Child 0 Input [2]: [quantity#74, list_price#75]
+Child 1 Input [2]: [quantity#80, list_price#81]
+Child 2 Input [2]: [quantity#86, list_price#87]
+
+(99) CometHashAggregate
+Input [2]: [quantity#74, list_price#75]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#74 as decimal(10,0)) * list_price#75))]
+
+(100) CometExchange
+Input [2]: [sum#88, count#89]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4]
+
+(101) CometHashAggregate
+Input [2]: [sum#88, count#89]
+Keys: []
+Functions [1]: [avg((cast(quantity#74 as decimal(10,0)) * list_price#75))]
+
+(102) ColumnarToRow [codegen id : 1]
+Input [1]: [average_sales#90]
+
+Subquery:2 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#41, [id=#42]
+* ColumnarToRow (106)
++- CometProject (105)
+ +- CometFilter (104)
+ +- CometScan parquet spark_catalog.default.date_dim (103)
+
+
+(103) CometScan parquet spark_catalog.default.date_dim
+Output [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)]
+ReadSchema: struct
+
+(104) CometFilter
+Input [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#93)) AND isnotnull(d_dom#94)) AND (d_year#92 = 2000)) AND (d_moy#93 = 12)) AND (d_dom#94 = 11))
+
+(105) CometProject
+Input [4]: [d_week_seq#91, d_year#92, d_moy#93, d_dom#94]
+Arguments: [d_week_seq#91], [d_week_seq#91]
+
+(106) ColumnarToRow [codegen id : 1]
+Input [1]: [d_week_seq#91]
+
+Subquery:3 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50]
+
+Subquery:4 Hosting operator id = 73 Hosting Expression = Subquery scalar-subquery#62, [id=#63]
+* ColumnarToRow (110)
++- CometProject (109)
+ +- CometFilter (108)
+ +- CometScan parquet spark_catalog.default.date_dim (107)
+
+
+(107) CometScan parquet spark_catalog.default.date_dim
+Output [4]: [d_week_seq#95, d_year#96, d_moy#97, d_dom#98]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)]
+ReadSchema: struct