From 55ba57d9cfaeb3facb569ad708193ddf8adb6660 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 19 Mar 2026 16:55:13 -0600 Subject: [PATCH 1/7] refactor: remove dead mutable buffer classes and BatchReader The mutable buffer code path (BatchReader, ConstantColumnReader, RowIndexColumnReader, MetadataColumnReader) was only used by the iceberg-java integration which is no longer supported. The main scan path (NativeBatchReader) uses immutable Arrow Java vectors via ArrowConstantColumnReader and ArrowRowIndexColumnReader instead. Remove the dead classes along with their JNI methods (setNull, setBoolean, setByte, setShort, setInt, setLong, setFloat, setDouble, setBinary, setDecimal, setPosition, setIndices, setIsDeleted) and corresponding Rust implementations. Also remove the hasScanUsingMutableBuffers check in EliminateRedundantTransitions since no scan path uses mutable buffers anymore. --- .../org/apache/comet/parquet/BatchReader.java | 534 ------------------ .../comet/parquet/ConstantColumnReader.java | 142 ----- .../parquet/IcebergCometBatchReader.java | 45 -- .../comet/parquet/MetadataColumnReader.java | 111 ---- .../java/org/apache/comet/parquet/Native.java | 33 -- .../comet/parquet/NativeBatchReader.java | 2 +- .../comet/parquet/RowIndexColumnReader.java | 49 -- .../org/apache/comet/parquet/TypeUtil.java | 5 +- .../comet/parquet/TestColumnReader.java | 139 ----- native/core/src/parquet/mod.rs | 227 +------- native/core/src/parquet/read/column.rs | 171 +----- native/core/src/parquet/read/mod.rs | 3 - .../rules/EliminateRedundantTransitions.scala | 26 +- 13 files changed, 11 insertions(+), 1476 deletions(-) delete mode 100644 common/src/main/java/org/apache/comet/parquet/BatchReader.java delete mode 100644 common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java delete mode 100644 common/src/main/java/org/apache/comet/parquet/IcebergCometBatchReader.java delete mode 100644 common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java delete mode 100644 common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java deleted file mode 100644 index 5a0bc9f6d3..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ /dev/null @@ -1,534 +0,0 @@ -/* - * 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 java.io.Closeable; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.*; - -import scala.Option; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.Preconditions; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.Type; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.comet.parquet.CometParquetReadSupport; -import org.apache.spark.sql.comet.shims.ShimTaskMetrics; -import org.apache.spark.sql.execution.datasources.PartitionedFile; -import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; -import org.apache.spark.sql.execution.metric.SQLMetric; -import org.apache.spark.sql.types.*; -import org.apache.spark.sql.vectorized.ColumnarBatch; -import org.apache.spark.util.AccumulatorV2; - -import org.apache.comet.CometConf; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.IcebergApi; -import org.apache.comet.shims.ShimBatchReader; -import org.apache.comet.shims.ShimFileFormat; -import org.apache.comet.vector.CometVector; - -/** - * A vectorized Parquet reader that reads a Parquet file in a batched fashion. - * - *

Example of how to use this: - * - *

- *   BatchReader reader = new BatchReader(parquetFile, batchSize);
- *   try {
- *     reader.init();
- *     while (reader.readBatch()) {
- *       ColumnarBatch batch = reader.currentBatch();
- *       // consume the batch
- *     }
- *   } finally { // resources associated with the reader should be released
- *     reader.close();
- *   }
- * 
- * - * @deprecated since 0.14.0. This class is kept for Iceberg compatibility only. - */ -@Deprecated -@IcebergApi -public class BatchReader extends RecordReader implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); - protected static final BufferAllocator ALLOCATOR = new RootAllocator(); - - private Configuration conf; - private int capacity; - private boolean isCaseSensitive; - private boolean useFieldId; - private boolean ignoreMissingIds; - private StructType partitionSchema; - private InternalRow partitionValues; - private PartitionedFile file; - protected Map metrics; - - private long rowsRead; - protected StructType sparkSchema; - private MessageType requestedSchema; - protected CometVector[] vectors; - protected AbstractColumnReader[] columnReaders; - private CometSchemaImporter importer; - protected ColumnarBatch currentBatch; - private FileReader fileReader; - private boolean[] missingColumns; - protected boolean isInitialized; - private ParquetMetadata footer; - - /** The total number of rows across all row groups of the input split. */ - private long totalRowCount; - - /** - * The total number of rows loaded so far, including all the rows from row groups that we've - * processed and the current row group. - */ - private long totalRowsLoaded; - - /** - * Whether the native scan should always return decimal represented by 128 bits, regardless of its - * precision. Normally, this should be true if native execution is enabled, since Arrow compute - * kernels doesn't support 32 and 64 bit decimals yet. - */ - private boolean useDecimal128; - - /** Whether to use the lazy materialization reader for reading columns. */ - private boolean useLazyMaterialization; - - /** - * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) - * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them - * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when - * seeing these dates/timestamps. - */ - private boolean useLegacyDateTimestamp; - - /** The TaskContext object for executing this task. */ - private TaskContext taskContext; - - public BatchReader() {} - - // Only for testing - public BatchReader(String file, int capacity) { - this(file, capacity, null, null); - } - - // Only for testing - public BatchReader( - String file, int capacity, StructType partitionSchema, InternalRow partitionValues) { - this(new Configuration(), file, capacity, partitionSchema, partitionValues); - } - - // Only for testing - public BatchReader( - Configuration conf, - String file, - int capacity, - StructType partitionSchema, - InternalRow partitionValues) { - conf.set("spark.sql.parquet.binaryAsString", "false"); - conf.set("spark.sql.parquet.int96AsTimestamp", "false"); - conf.set("spark.sql.caseSensitive", "false"); - conf.set("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); - conf.set("spark.sql.legacy.parquet.nanosAsLong", "false"); - - this.conf = conf; - this.capacity = capacity; - this.isCaseSensitive = false; - this.useFieldId = false; - this.ignoreMissingIds = false; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - - this.file = ShimBatchReader.newPartitionedFile(partitionValues, file); - this.metrics = new HashMap<>(); - - this.taskContext = TaskContext$.MODULE$.get(); - } - - /** - * @see Comet Issue #2079 - */ - @IcebergApi - public BatchReader(AbstractColumnReader[] columnReaders) { - // Todo: set useDecimal128 and useLazyMaterialization - int numColumns = columnReaders.length; - this.columnReaders = new AbstractColumnReader[numColumns]; - vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); - // This constructor is used by Iceberg only. The columnReaders are - // initialized in Iceberg, so no need to call the init() - isInitialized = true; - this.taskContext = TaskContext$.MODULE$.get(); - this.metrics = new HashMap<>(); - } - - BatchReader( - Configuration conf, - PartitionedFile inputSplit, - ParquetMetadata footer, - int capacity, - StructType sparkSchema, - boolean isCaseSensitive, - boolean useFieldId, - boolean ignoreMissingIds, - boolean useLegacyDateTimestamp, - StructType partitionSchema, - InternalRow partitionValues, - Map metrics) { - this.conf = conf; - this.capacity = capacity; - this.sparkSchema = sparkSchema; - this.isCaseSensitive = isCaseSensitive; - this.useFieldId = useFieldId; - this.ignoreMissingIds = ignoreMissingIds; - this.useLegacyDateTimestamp = useLegacyDateTimestamp; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - this.file = inputSplit; - this.footer = footer; - this.metrics = metrics; - this.taskContext = TaskContext$.MODULE$.get(); - } - - /** - * Initialize this reader. The reason we don't do it in the constructor is that we want to close - * any resource hold by this reader when error happens during the initialization. - */ - public void init() throws URISyntaxException, IOException { - useDecimal128 = - conf.getBoolean( - CometConf.COMET_USE_DECIMAL_128().key(), - (Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get()); - useLazyMaterialization = - conf.getBoolean( - CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), - (Boolean) CometConf.COMET_USE_LAZY_MATERIALIZATION().defaultValue().get()); - - long start = file.start(); - long length = file.length(); - String filePath = file.filePath().toString(); - - ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath)); - - if (start >= 0 && length >= 0) { - builder = builder.withRange(start, start + length); - } - ParquetReadOptions readOptions = builder.build(); - - // TODO: enable off-heap buffer when they are ready - ReadOptions cometReadOptions = ReadOptions.builder(conf).build(); - - Path path = new Path(new URI(filePath)); - fileReader = - new FileReader( - CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics); - requestedSchema = fileReader.getFileMetaData().getSchema(); - MessageType fileSchema = requestedSchema; - - if (sparkSchema == null) { - sparkSchema = new ParquetToSparkSchemaConverter(conf).convert(requestedSchema); - } else { - requestedSchema = - CometParquetReadSupport.clipParquetSchema( - requestedSchema, sparkSchema, isCaseSensitive, useFieldId, ignoreMissingIds); - if (requestedSchema.getFieldCount() != sparkSchema.size()) { - throw new IllegalArgumentException( - String.format( - "Spark schema has %d columns while " + "Parquet schema has %d columns", - sparkSchema.size(), requestedSchema.getColumns().size())); - } - } - - totalRowCount = fileReader.getRecordCount(); - List columns = requestedSchema.getColumns(); - int numColumns = columns.size(); - if (partitionSchema != null) numColumns += partitionSchema.size(); - columnReaders = new AbstractColumnReader[numColumns]; - - // Initialize missing columns and use null vectors for them - missingColumns = new boolean[columns.size()]; - List paths = requestedSchema.getPaths(); - // We do not need the column index of the row index; but this method has the - // side effect of throwing an exception if a column with the same name is - // found which we do want (spark unit tests explicitly test for that). - ShimFileFormat.findRowIndexColumnIndexInSchema(sparkSchema); - StructField[] nonPartitionFields = sparkSchema.fields(); - for (int i = 0; i < requestedSchema.getFieldCount(); i++) { - Type t = requestedSchema.getFields().get(i); - Preconditions.checkState( - t.isPrimitive() && !t.isRepetition(Type.Repetition.REPEATED), - "Complex type is not supported"); - String[] colPath = paths.get(i); - if (nonPartitionFields[i].name().equals(ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME())) { - // Values of ROW_INDEX_TEMPORARY_COLUMN_NAME column are always populated with - // generated row indexes, rather than read from the file. - // TODO(SPARK-40059): Allow users to include columns named - // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas. - long[] rowIndices = fileReader.getRowIndices(); - columnReaders[i] = new RowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); - missingColumns[i] = true; - } else if (fileSchema.containsPath(colPath)) { - ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); - if (!fd.equals(columns.get(i))) { - throw new UnsupportedOperationException("Schema evolution is not supported"); - } - missingColumns[i] = false; - } else { - if (columns.get(i).getMaxDefinitionLevel() == 0) { - throw new IOException( - "Required column '" - + Arrays.toString(colPath) - + "' is missing" - + " in data file " - + filePath); - } - ConstantColumnReader reader = - new ConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); - columnReaders[i] = reader; - missingColumns[i] = true; - } - } - - // Initialize constant readers for partition columns - if (partitionSchema != null) { - StructField[] partitionFields = partitionSchema.fields(); - for (int i = columns.size(); i < columnReaders.length; i++) { - int fieldIndex = i - columns.size(); - StructField field = partitionFields[fieldIndex]; - ConstantColumnReader reader = - new ConstantColumnReader(field, capacity, partitionValues, fieldIndex, useDecimal128); - columnReaders[i] = reader; - } - } - - vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); - fileReader.setRequestedSchema(requestedSchema.getColumns()); - - // For test purpose only - // If the last external accumulator is `NumRowGroupsAccumulator`, the row group number to read - // will be updated to the accumulator. So we can check if the row groups are filtered or not - // in test case. - // Note that this tries to get thread local TaskContext object, if this is called at other - // thread, it won't update the accumulator. - if (taskContext != null) { - Option> accu = - ShimTaskMetrics.getTaskAccumulator(taskContext.taskMetrics()); - if (accu.isDefined() && accu.get().getClass().getSimpleName().equals("NumRowGroupsAcc")) { - @SuppressWarnings("unchecked") - AccumulatorV2 intAccum = (AccumulatorV2) accu.get(); - intAccum.add(fileReader.getRowGroups().size()); - } - } - - isInitialized = true; - } - - /** - * @see Comet Issue #2079 - */ - @IcebergApi - public void setSparkSchema(StructType schema) { - this.sparkSchema = schema; - } - - /** - * @see Comet Issue #2079 - */ - @IcebergApi - 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 (float) rowsRead / totalRowCount; - } - - /** - * Returns the current columnar batch being read. - * - *

Note that this must be called AFTER {@link BatchReader#nextBatch()}. - */ - public ColumnarBatch currentBatch() { - return currentBatch; - } - - /** - * Loads the next batch of rows. - * - * @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; - boolean hasMore; - - try { - hasMore = loadNextRowGroupIfNecessary(); - } catch (RuntimeException e) { - // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. - throw e; - } catch (Throwable e) { - throw new IOException(e); - } - - if (!hasMore) return false; - int batchSize = (int) Math.min(capacity, totalRowsLoaded - rowsRead); - - return nextBatch(batchSize); - } - - @IcebergApi - public boolean nextBatch(int batchSize) { - long totalDecodeTime = 0, totalLoadTime = 0; - for (int i = 0; i < columnReaders.length; i++) { - AbstractColumnReader reader = columnReaders[i]; - long startNs = System.nanoTime(); - reader.readBatch(batchSize); - totalDecodeTime += System.nanoTime() - startNs; - startNs = System.nanoTime(); - vectors[i] = reader.currentBatch(); - totalLoadTime += System.nanoTime() - startNs; - } - - 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); - rowsRead += batchSize; - return true; - } - - @IcebergApi - @Override - public void close() throws IOException { - if (columnReaders != null) { - for (AbstractColumnReader reader : columnReaders) { - if (reader != null) { - reader.close(); - } - } - } - if (fileReader != null) { - fileReader.close(); - fileReader = null; - } - if (importer != null) { - importer.close(); - importer = null; - } - } - - private boolean loadNextRowGroupIfNecessary() throws Throwable { - // More rows can be read from loaded row group. No need to load next one. - if (rowsRead != totalRowsLoaded) return true; - - SQLMetric rowGroupTimeMetric = metrics.get("ParquetLoadRowGroupTime"); - SQLMetric numRowGroupsMetric = metrics.get("ParquetRowGroups"); - long startNs = System.nanoTime(); - - PageReadStore rowGroupReader = fileReader.readNextRowGroup(); - - if (rowGroupTimeMetric != null) { - rowGroupTimeMetric.add(System.nanoTime() - startNs); - } - if (rowGroupReader == null) { - return false; - } - if (numRowGroupsMetric != null) { - numRowGroupsMetric.add(1); - } - - if (importer != null) importer.close(); - importer = new CometSchemaImporter(ALLOCATOR); - - List columns = requestedSchema.getColumns(); - for (int i = 0; i < columns.size(); i++) { - if (missingColumns[i]) continue; - if (columnReaders[i] != null) columnReaders[i].close(); - // TODO: handle tz, datetime & int96 rebase - // TODO: consider passing page reader via ctor - however we need to fix the shading issue - // from Iceberg side. - DataType dataType = sparkSchema.fields()[i].dataType(); - ColumnReader reader = - Utils.getColumnReader( - dataType, - columns.get(i), - importer, - capacity, - useDecimal128, - useLazyMaterialization, - useLegacyDateTimestamp); - reader.setPageReader(rowGroupReader.getPageReader(columns.get(i))); - columnReaders[i] = reader; - } - totalRowsLoaded += rowGroupReader.getRowCount(); - return true; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java deleted file mode 100644 index 1d4ec84c2c..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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 java.math.BigInteger; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns; -import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.UTF8String; - -import org.apache.comet.IcebergApi; - -/** - * A column reader that always return constant vectors. Used for reading partition columns, for - * instance. - */ -@IcebergApi -public class ConstantColumnReader extends MetadataColumnReader { - /** Whether all the values in this constant column are nulls */ - private boolean isNull; - - /** The constant value in the format of Object that are used to initialize this column reader. */ - private Object value; - - ConstantColumnReader(StructField field, int batchSize, boolean useDecimal128) { - this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); - this.value = - ResolveDefaultColumns.getExistenceDefaultValues(new StructType(new StructField[] {field}))[ - 0]; - init(value); - } - - ConstantColumnReader( - StructField field, int batchSize, InternalRow values, int index, boolean useDecimal128) { - this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); - init(values, index); - } - - /** - * @see Comet Issue #2079 - */ - @IcebergApi - public ConstantColumnReader( - DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { - super(type, descriptor, useDecimal128, true); - this.value = value; - } - - // Used by Iceberg - @IcebergApi - public ConstantColumnReader( - DataType type, ParquetColumnSpec spec, Object value, boolean useDecimal128) { - super(type, spec, useDecimal128, true); - this.value = value; - } - - ConstantColumnReader( - DataType type, ColumnDescriptor descriptor, int batchSize, boolean useDecimal128) { - super(type, descriptor, useDecimal128, true); - this.batchSize = batchSize; - initNative(); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - init(value); - } - - @Override - public void readBatch(int total) { - super.readBatch(total); - if (isNull) setNumNulls(total); - } - - private void init(InternalRow values, int index) { - Object value = values.get(index, type); - init(value); - } - - private void init(Object value) { - if (value == null) { - Native.setNull(nativeHandle); - isNull = true; - } else if (type == DataTypes.BooleanType) { - Native.setBoolean(nativeHandle, (boolean) value); - } else if (type == DataTypes.ByteType) { - Native.setByte(nativeHandle, (byte) value); - } else if (type == DataTypes.ShortType) { - Native.setShort(nativeHandle, (short) value); - } else if (type == DataTypes.IntegerType) { - Native.setInt(nativeHandle, (int) value); - } else if (type == DataTypes.LongType) { - Native.setLong(nativeHandle, (long) value); - } else if (type == DataTypes.FloatType) { - Native.setFloat(nativeHandle, (float) value); - } else if (type == DataTypes.DoubleType) { - Native.setDouble(nativeHandle, (double) value); - } else if (type == DataTypes.BinaryType) { - Native.setBinary(nativeHandle, (byte[]) value); - } else if (type == DataTypes.StringType) { - Native.setBinary(nativeHandle, ((UTF8String) value).getBytes()); - } else if (type == DataTypes.DateType) { - Native.setInt(nativeHandle, (int) value); - } else if (type == DataTypes.TimestampType || type == TimestampNTZType$.MODULE$) { - Native.setLong(nativeHandle, (long) value); - } else if (type instanceof DecimalType) { - DecimalType dt = (DecimalType) type; - Decimal d = (Decimal) value; - if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { - Native.setInt(nativeHandle, ((int) d.toUnscaledLong())); - } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { - Native.setLong(nativeHandle, d.toUnscaledLong()); - } else { - final BigInteger integer = d.toJavaBigDecimal().unscaledValue(); - byte[] bytes = integer.toByteArray(); - Native.setDecimal(nativeHandle, bytes); - } - } else { - throw new UnsupportedOperationException("Unsupported Spark type: " + type); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/IcebergCometBatchReader.java b/common/src/main/java/org/apache/comet/parquet/IcebergCometBatchReader.java deleted file mode 100644 index bd66f2deab..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/IcebergCometBatchReader.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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 java.util.HashMap; - -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -import org.apache.comet.IcebergApi; -import org.apache.comet.vector.CometVector; - -/** This class is a public interface used by Apache Iceberg to read batches using Comet */ -@IcebergApi -public class IcebergCometBatchReader extends BatchReader { - public IcebergCometBatchReader(int numColumns, StructType schema) { - this.columnReaders = new AbstractColumnReader[numColumns]; - this.vectors = new CometVector[numColumns]; - this.currentBatch = new ColumnarBatch(vectors); - this.metrics = new HashMap<>(); - this.sparkSchema = schema; - } - - public void init(AbstractColumnReader[] columnReaders) { - this.columnReaders = columnReaders; - this.isInitialized = true; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java deleted file mode 100644 index ace1ab4164..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.apache.arrow.c.ArrowArray; -import org.apache.arrow.c.ArrowSchema; -import org.apache.arrow.c.Data; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.FieldVector; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.types.DataType; - -import org.apache.comet.IcebergApi; -import org.apache.comet.vector.CometPlainVector; -import org.apache.comet.vector.CometVector; - -/** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ -@IcebergApi -public class MetadataColumnReader extends AbstractColumnReader { - private final BufferAllocator allocator = new RootAllocator(); - - private CometVector vector; - - private ArrowArray array = null; - private ArrowSchema schema = null; - - private boolean isConstant; - - /** - * @see Comet Issue #2079 - */ - @IcebergApi - public MetadataColumnReader( - DataType type, ColumnDescriptor descriptor, boolean useDecimal128, boolean isConstant) { - // TODO: should we handle legacy dates & timestamps for metadata columns? - super(type, descriptor, useDecimal128, false); - - this.isConstant = isConstant; - } - - // Used by Iceberg - @IcebergApi - public MetadataColumnReader( - DataType type, ParquetColumnSpec spec, boolean useDecimal128, boolean isConstant) { - // TODO: should we handle legacy dates & timestamps for metadata columns? - super(type, Utils.buildColumnDescriptor(spec), useDecimal128, false); - - this.isConstant = isConstant; - } - - @Override - public void setBatchSize(int batchSize) { - close(); - super.setBatchSize(batchSize); - } - - @IcebergApi - @Override - public void readBatch(int total) { - if (vector == null) { - array = ArrowArray.allocateNew(allocator); - schema = ArrowSchema.allocateNew(allocator); - - long arrayAddr = array.memoryAddress(); - long schemaAddr = schema.memoryAddress(); - - Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); - FieldVector fieldVector = Data.importVector(allocator, array, schema, null); - vector = new CometPlainVector(fieldVector, useDecimal128, false, isConstant); - } - - vector.setNumValues(total); - } - - void setNumNulls(int total) { - vector.setNumNulls(total); - } - - @IcebergApi - @Override - public CometVector currentBatch() { - return vector; - } - - @Override - public void close() { - if (vector != null) { - vector.close(); - vector = null; - } - super.close(); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java index babd0d392c..fb9cedc1e2 100644 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -200,39 +200,6 @@ public static native void setPageV2( */ public static native void currentBatch(long handle, long arrayAddr, long schemaAddr); - /** Set methods to set a constant value for the reader, so it'll return constant vectors */ - public static native void setNull(long handle); - - public static native void setBoolean(long handle, boolean value); - - public static native void setByte(long handle, byte value); - - public static native void setShort(long handle, short value); - - public static native void setInt(long handle, int value); - - public static native void setLong(long handle, long value); - - public static native void setFloat(long handle, float value); - - public static native void setDouble(long handle, double value); - - public static native void setBinary(long handle, byte[] value); - - /** Set decimal backed by FixedLengthByteArray */ - public static native void setDecimal(long handle, byte[] value); - - /** Set position of row index vector for Iceberg Metadata Column */ - @IcebergApi - public static native void setPosition(long handle, long value, int size); - - /** Set row index vector for Spark row index metadata column and return vector size */ - public static native int setIndices(long handle, long offset, int size, long[] indices); - - /** Set deleted info for Iceberg Metadata Column */ - @IcebergApi - public static native void setIsDeleted(long handle, boolean[] isDeleted); - /** * Closes the native Parquet column reader and releases all resources associated with it. * diff --git a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java b/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java index 32edcb2640..9413b9316d 100644 --- a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java @@ -89,7 +89,7 @@ *

Example of how to use this: * *

- *   BatchReader reader = new BatchReader(parquetFile, batchSize);
+ *   NativeBatchReader reader = new NativeBatchReader(parquetFile, batchSize);
  *   try {
  *     reader.init();
  *     while (reader.readBatch()) {
diff --git a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java
deleted file mode 100644
index 46e6ee67f5..0000000000
--- a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.apache.spark.sql.types.*;
-
-/**
- * A column reader that returns the row index vector. Used for reading row index metadata column for
- * Spark 3.4+. The row index can be accessed by {@code _tmp_metadata_row_index} column.
- */
-public class RowIndexColumnReader extends MetadataColumnReader {
-  /** The row indices that are used to initialize this column reader. */
-  private final long[] indices;
-
-  /** The current number of indices to skip reading from {@code indices}. */
-  private long offset;
-
-  public RowIndexColumnReader(StructField field, int batchSize, long[] indices) {
-    super(field.dataType(), TypeUtil.convertToParquet(field), false, false);
-    this.indices = indices;
-    setBatchSize(batchSize);
-  }
-
-  @Override
-  public void readBatch(int total) {
-    Native.resetBatch(nativeHandle);
-    int count = Native.setIndices(nativeHandle, offset, total, indices);
-    offset += count;
-
-    super.readBatch(count);
-  }
-}
diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java
index cb5f4997dd..87cecdc65d 100644
--- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java
+++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java
@@ -57,9 +57,8 @@ public static ColumnDescriptor convertToParquet(StructField field) {
     DataType type = field.dataType();
 
     Types.PrimitiveBuilder builder = null;
-    // Only partition column can be `NullType`, which also uses `ConstantColumnReader`. Here we
-    // piggy-back onto Parquet boolean type for constant vector of null values, we don't really
-    // care what Parquet type it is.
+    // Only partition column can be `NullType`. Here we piggy-back onto Parquet boolean type
+    // for constant vector of null values, we don't really care what Parquet type it is.
     if (type == DataTypes.BooleanType || type == DataTypes.NullType) {
       builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition);
     } else if (type == DataTypes.IntegerType || type instanceof YearMonthIntervalType) {
diff --git a/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java
index 32accae8e5..2052222ecd 100644
--- a/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java
+++ b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java
@@ -19,13 +19,6 @@
 
 package org.apache.comet.parquet;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.function.BiFunction;
-
 import org.junit.Test;
 
 import org.apache.arrow.memory.BufferAllocator;
@@ -34,145 +27,13 @@
 import org.apache.arrow.vector.IntVector;
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.VarBinaryVector;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.types.*;
-import org.apache.spark.sql.vectorized.ColumnVector;
 
 import org.apache.comet.vector.CometPlainVector;
 import org.apache.comet.vector.CometVector;
 
-import static org.apache.spark.sql.types.DataTypes.*;
 import static org.junit.Assert.*;
-import static scala.jdk.javaapi.CollectionConverters.*;
 
-@SuppressWarnings("unchecked")
 public class TestColumnReader {
-  private static final int BATCH_SIZE = 1024;
-  private static final List TYPES =
-      Arrays.asList(
-          BooleanType,
-          ByteType,
-          ShortType,
-          IntegerType,
-          LongType,
-          FloatType,
-          DoubleType,
-          BinaryType,
-          DecimalType.apply(5, 2),
-          DecimalType.apply(18, 10),
-          DecimalType.apply(19, 5));
-  private static final List VALUES =
-      Arrays.asList(
-          true,
-          (byte) 42,
-          (short) 100,
-          1000,
-          (long) 10000,
-          (float) 3.14,
-          3.1415926,
-          new byte[] {1, 2, 3, 4, 5, 6, 7, 8},
-          Decimal.apply("123.45"),
-          Decimal.apply("00.0123456789"),
-          Decimal.apply("-001234.56789"));
-  private static final List> GETTERS =
-      Arrays.asList(
-          ColumnVector::getBoolean,
-          ColumnVector::getByte,
-          ColumnVector::getShort,
-          ColumnVector::getInt,
-          ColumnVector::getLong,
-          ColumnVector::getFloat,
-          ColumnVector::getDouble,
-          ColumnVector::getBinary,
-          (v, i) -> v.getDecimal(i, 5, 2),
-          (v, i) -> v.getDecimal(i, 18, 10),
-          (v, i) -> v.getDecimal(i, 19, 5));
-
-  @Test
-  public void testConstantVectors() {
-    for (int i = 0; i < TYPES.size(); i++) {
-      DataType type = TYPES.get(i);
-      StructField field = StructField.apply("f", type, false, null);
-
-      List values = Collections.singletonList(VALUES.get(i));
-      InternalRow row = GenericInternalRow.apply(asScala(values).toSeq());
-      ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true);
-      reader.readBatch(BATCH_SIZE);
-      CometVector vector = reader.currentBatch();
-      assertEquals(BATCH_SIZE, vector.numValues());
-      assertEquals(0, vector.numNulls());
-      for (int j = 0; j < BATCH_SIZE; j++) {
-        if (TYPES.get(i) == BinaryType || TYPES.get(i) == StringType) {
-          assertArrayEquals((byte[]) VALUES.get(i), (byte[]) GETTERS.get(i).apply(vector, j));
-        } else {
-          assertEquals(VALUES.get(i), GETTERS.get(i).apply(vector, j));
-        }
-      }
-
-      // Test null values too
-      row.setNullAt(0);
-      reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true);
-      reader.readBatch(BATCH_SIZE);
-      vector = reader.currentBatch();
-      assertEquals(BATCH_SIZE, vector.numValues());
-      assertEquals(BATCH_SIZE, vector.numNulls());
-      for (int j = 0; j < BATCH_SIZE; j++) {
-        assertTrue(vector.isNullAt(j));
-      }
-    }
-
-    if (org.apache.spark.package$.MODULE$.SPARK_VERSION_SHORT().compareTo("3.4") >= 0) {
-      Metadata meta = new MetadataBuilder().putString("EXISTS_DEFAULT", "123").build();
-      StructField field = StructField.apply("f", LongType, false, meta);
-      ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, true);
-      reader.readBatch(BATCH_SIZE);
-      CometVector vector = reader.currentBatch();
-
-      assertEquals(BATCH_SIZE, vector.numValues());
-      assertEquals(0, vector.numNulls());
-      for (int j = 0; j < BATCH_SIZE; j++) {
-        assertEquals(123, vector.getLong(j));
-      }
-    }
-  }
-
-  @Test
-  public void testRowIndexColumnVectors() {
-    StructField field = StructField.apply("f", LongType, false, null);
-    int bigBatchSize = BATCH_SIZE * 2;
-    int step = 4;
-    int batchSize = bigBatchSize / step;
-    long[] indices = new long[step * 2];
-    List expected = new ArrayList<>();
-
-    long idx = 0, len = 0;
-    for (int i = 0; i < step; i++) {
-      idx = ThreadLocalRandom.current().nextLong(idx + len, Long.MAX_VALUE);
-      indices[i * 2] = idx;
-      len = ThreadLocalRandom.current().nextLong(Long.max(bigBatchSize - expected.size(), 0));
-      indices[i * 2 + 1] = len;
-      for (int j = 0; j < len; j++) {
-        expected.add(idx + j);
-      }
-    }
-
-    RowIndexColumnReader reader = new RowIndexColumnReader(field, BATCH_SIZE, indices);
-    for (int i = 0; i < step; i++) {
-      reader.readBatch(batchSize);
-      CometVector vector = reader.currentBatch();
-      assertEquals(
-          Integer.min(batchSize, Integer.max(expected.size() - i * batchSize, 0)),
-          vector.numValues());
-      assertEquals(0, vector.numNulls());
-      for (int j = 0; j < vector.numValues(); j++) {
-        assertEquals((long) expected.get(i * batchSize + j), vector.getLong(j));
-      }
-    }
-
-    reader.close();
-  }
-
   @Test
   public void testIsFixedLength() {
     BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE);
diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs
index f2b0e80ab2..237c6c7383 100644
--- a/native/core/src/parquet/mod.rs
+++ b/native/core/src/parquet/mod.rs
@@ -42,7 +42,7 @@ use arrow::ffi::FFI_ArrowArray;
 use jni::JNIEnv;
 use jni::{
     objects::{GlobalRef, JByteBuffer, JClass},
-    sys::{jboolean, jbyte, jdouble, jfloat, jint, jlong, jshort},
+    sys::{jboolean, jint, jlong},
 };
 
 use self::util::jni::TypePromotionInfo;
@@ -66,7 +66,7 @@ use datafusion::physical_plan::ExecutionPlan;
 use datafusion::prelude::{SessionConfig, SessionContext};
 use futures::{poll, StreamExt};
 use jni::objects::{
-    JBooleanArray, JByteArray, JLongArray, JMap, JObject, JObjectArray, JString, ReleaseMode,
+    JByteArray, JLongArray, JMap, JObject, JObjectArray, JString, ReleaseMode,
 };
 use jni::sys::{jintArray, JNI_FALSE};
 use object_store::path::Path;
@@ -275,229 +275,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setPageV2(
     })
 }
 
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setNull(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_null();
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setBoolean(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jboolean,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_boolean(value != 0);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setByte(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jbyte,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setShort(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jshort,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setInt(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jint,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setLong(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jlong,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setFloat(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jfloat,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setDouble(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jdouble,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_fixed::(value);
-        Ok(())
-    })
-}
-
-/// # Safety
-/// This function is inheritly unsafe since it deals with raw pointers passed from JNI.
-#[no_mangle]
-pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setBinary(
-    e: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: JByteArray,
-) {
-    try_unwrap_or_throw(&e, |env| {
-        let reader = get_reader(handle)?;
-
-        let len = env.get_array_length(&value)?;
-        let mut buffer = MutableBuffer::from_len_zeroed(len as usize);
-        env.get_byte_array_region(&value, 0, from_u8_slice(buffer.as_slice_mut()))?;
-        reader.set_binary(buffer);
-        Ok(())
-    })
-}
-
-/// # Safety
-/// This function is inheritly unsafe since it deals with raw pointers passed from JNI.
-#[no_mangle]
-pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setDecimal(
-    e: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: JByteArray,
-) {
-    try_unwrap_or_throw(&e, |env| {
-        let reader = get_reader(handle)?;
-
-        let len = env.get_array_length(&value)?;
-        let mut buffer = MutableBuffer::from_len_zeroed(len as usize);
-        env.get_byte_array_region(&value, 0, from_u8_slice(buffer.as_slice_mut()))?;
-        reader.set_decimal_flba(buffer);
-        Ok(())
-    })
-}
-
-#[no_mangle]
-pub extern "system" fn Java_org_apache_comet_parquet_Native_setPosition(
-    env: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    value: jlong,
-    size: jint,
-) {
-    try_unwrap_or_throw(&env, |_| {
-        let reader = get_reader(handle)?;
-        reader.set_position(value, size as usize);
-        Ok(())
-    })
-}
-
-/// # Safety
-/// This function is inheritly unsafe since it deals with raw pointers passed from JNI.
-#[no_mangle]
-pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setIndices(
-    e: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    offset: jlong,
-    batch_size: jint,
-    indices: JLongArray,
-) -> jlong {
-    try_unwrap_or_throw(&e, |mut env| {
-        let reader = get_reader(handle)?;
-        let indices = unsafe { env.get_array_elements(&indices, ReleaseMode::NoCopyBack)? };
-        let len = indices.len();
-        // paris alternately contains start index and length of continuous indices
-        let pairs = unsafe { core::slice::from_raw_parts_mut(indices.as_ptr(), len) };
-        let mut skipped = 0;
-        let mut filled = 0;
-        for i in (0..len).step_by(2) {
-            let index = pairs[i];
-            let count = pairs[i + 1];
-            let skip = std::cmp::min(count, offset - skipped);
-            skipped += skip;
-            if count == skip {
-                continue;
-            } else if batch_size as i64 == filled {
-                break;
-            }
-            let count = std::cmp::min(count - skip, batch_size as i64 - filled);
-            filled += count;
-            reader.set_position(index + skip, count as usize);
-        }
-        Ok(filled)
-    })
-}
-
-/// # Safety
-/// This function is inheritly unsafe since it deals with raw pointers passed from JNI.
-#[no_mangle]
-pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setIsDeleted(
-    e: JNIEnv,
-    _jclass: JClass,
-    handle: jlong,
-    is_deleted: JBooleanArray,
-) {
-    try_unwrap_or_throw(&e, |env| {
-        let reader = get_reader(handle)?;
-
-        let len = env.get_array_length(&is_deleted)?;
-        let mut buffer = MutableBuffer::from_len_zeroed(len as usize);
-        env.get_boolean_array_region(&is_deleted, 0, buffer.as_slice_mut())?;
-        reader.set_is_deleted(buffer);
-        Ok(())
-    })
-}
-
 #[no_mangle]
 pub extern "system" fn Java_org_apache_comet_parquet_Native_resetBatch(
     env: JNIEnv,
diff --git a/native/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs
index c2b64bba51..cdbb76bbf0 100644
--- a/native/core/src/parquet/read/column.rs
+++ b/native/core/src/parquet/read/column.rs
@@ -19,8 +19,8 @@ use std::{marker::PhantomData, sync::Arc};
 
 use arrow::{
     array::ArrayData,
-    buffer::{Buffer, MutableBuffer},
-    datatypes::{ArrowNativeType, DataType as ArrowDataType, TimeUnit},
+    buffer::Buffer,
+    datatypes::{DataType as ArrowDataType, TimeUnit},
 };
 
 use parquet::{
@@ -28,9 +28,7 @@ use parquet::{
     schema::types::{ColumnDescPtr, ColumnDescriptor},
 };
 
-use crate::parquet::{
-    data_type::*, read::DECIMAL_BYTE_WIDTH, util::jni::TypePromotionInfo, ParquetMutableVector,
-};
+use crate::parquet::{data_type::*, util::jni::TypePromotionInfo, ParquetMutableVector};
 
 use super::{
     levels::LevelDecoder,
@@ -38,7 +36,7 @@ use super::{
     ReadOptions,
 };
 
-use crate::common::{bit, bit::log2};
+use crate::common::bit::log2;
 use crate::execution::operators::ExecutionError;
 
 /// Maximum number of decimal digits an i32 can represent
@@ -575,41 +573,6 @@ impl ColumnReader {
         )
     }
 
-    #[inline]
-    pub fn set_null(&mut self) {
-        make_func_mut!(self, set_null)
-    }
-
-    #[inline]
-    pub fn set_boolean(&mut self, value: bool) {
-        make_func_mut!(self, set_boolean, value)
-    }
-
-    #[inline]
-    pub fn set_fixed(&mut self, value: U) {
-        make_func_mut!(self, set_fixed, value)
-    }
-
-    #[inline]
-    pub fn set_binary(&mut self, value: MutableBuffer) {
-        make_func_mut!(self, set_binary, value)
-    }
-
-    #[inline]
-    pub fn set_decimal_flba(&mut self, value: MutableBuffer) {
-        make_func_mut!(self, set_decimal_flba, value)
-    }
-
-    #[inline]
-    pub fn set_position(&mut self, value: i64, size: usize) {
-        make_func_mut!(self, set_position, value, size)
-    }
-
-    #[inline]
-    pub fn set_is_deleted(&mut self, value: MutableBuffer) {
-        make_func_mut!(self, set_is_deleted, value)
-    }
-
     #[inline]
     pub fn reset_batch(&mut self) {
         make_func_mut!(self, reset_batch)
@@ -647,9 +610,6 @@ pub struct TypedColumnReader {
     capacity: usize,
     /// Number of bits used to represent one value in Parquet.
     bit_width: usize,
-    /// Whether this is a constant column reader (always return constant vector).
-    is_const: bool,
-
     // Options for reading Parquet
     read_options: ReadOptions,
 
@@ -676,7 +636,6 @@ impl TypedColumnReader {
             vector,
             capacity,
             bit_width,
-            is_const: false,
             read_options,
             _phantom: PhantomData,
         }
@@ -857,128 +816,6 @@ impl TypedColumnReader {
         self.value_decoder = Some(value_decoder);
     }
 
-    /// Sets all values in the vector of this column reader to be null.
-    pub fn set_null(&mut self) {
-        self.check_const("set_null");
-        self.vector.put_nulls(self.capacity);
-    }
-
-    /// Sets all values in the vector of this column reader to be `value`.
-    pub fn set_boolean(&mut self, value: bool) {
-        self.check_const("set_boolean");
-        if value {
-            let dst = self.vector.value_buffer.as_slice_mut();
-            bit::set_bits(dst, 0, self.capacity);
-        }
-        self.vector.num_values += self.capacity;
-    }
-
-    /// Sets all values in the vector of this column reader to be `value`.
-    pub fn set_fixed(&mut self, value: U) {
-        self.check_const("set_fixed");
-        let type_size = std::mem::size_of::();
-
-        let mut offset = 0;
-        for _ in 0..self.capacity {
-            bit::memcpy_value(&value, type_size, &mut self.vector.value_buffer[offset..]);
-            offset += type_size;
-        }
-        self.vector.num_values += self.capacity;
-    }
-
-    /// Sets all values in the vector of this column reader to be binary represented by `buffer`.
-    pub fn set_binary(&mut self, buffer: MutableBuffer) {
-        self.check_const("set_binary");
-
-        // TODO: consider using dictionary here
-
-        let len = buffer.len();
-        let total_len = len * self.capacity;
-        let offset_buf = self.vector.value_buffer.as_slice_mut();
-        let child_vector = &mut self.vector.children[0];
-        let value_buf = &mut child_vector.value_buffer;
-
-        value_buf.resize(total_len);
-
-        let mut value_buf_offset = 0;
-        let mut offset_buf_offset = 4;
-        for _ in 0..self.capacity {
-            bit::memcpy(&buffer, &mut value_buf.as_slice_mut()[value_buf_offset..]);
-            value_buf_offset += len;
-
-            bit::memcpy_value(
-                &(value_buf_offset as i32),
-                4,
-                &mut offset_buf[offset_buf_offset..],
-            );
-            offset_buf_offset += 4;
-        }
-        self.vector.num_values += self.capacity;
-    }
-
-    /// Sets all values in the vector of this column reader to be decimal represented by `buffer`.
-    pub fn set_decimal_flba(&mut self, buffer: MutableBuffer) {
-        self.check_const("set_decimal_flba");
-
-        // TODO: consider using dictionary here
-
-        let len = buffer.len();
-        let mut bytes: [u8; DECIMAL_BYTE_WIDTH] = [0; DECIMAL_BYTE_WIDTH];
-
-        for i in 0..len {
-            bytes[len - i - 1] = buffer[i];
-        }
-        if bytes[len - 1] & 0x80 == 0x80 {
-            bytes[len..DECIMAL_BYTE_WIDTH].fill(0xff);
-        }
-
-        let mut offset = 0;
-        for _ in 0..self.capacity {
-            bit::memcpy(&bytes, &mut self.vector.value_buffer[offset..]);
-            offset += DECIMAL_BYTE_WIDTH;
-        }
-        self.vector.num_values += self.capacity;
-    }
-
-    /// Sets position values of this column reader to the vector starting from `value`.
-    pub fn set_position(&mut self, value: i64, size: usize) {
-        let i64_size = std::mem::size_of::();
-
-        let mut offset = self.vector.num_values * i64_size;
-        for i in value..(value + size as i64) {
-            // TODO: is it better to convert self.value_buffer to &mut [i64] and for-loop update?
-            bit::memcpy_value(&i, i64_size, &mut self.vector.value_buffer[offset..]);
-            offset += i64_size;
-        }
-        self.vector.num_values += size;
-    }
-
-    /// Sets the values in the vector of this column reader to be a boolean array represented
-    /// by `buffer`.
-    pub fn set_is_deleted(&mut self, buffer: MutableBuffer) {
-        let len = buffer.len();
-        let dst = self.vector.value_buffer.as_slice_mut();
-        for i in 0..len {
-            if buffer[i] == 1 {
-                bit::set_bit(dst, i);
-            } else if buffer[i] == 0 {
-                bit::unset_bit(dst, i);
-            }
-        }
-        self.vector.num_values += len;
-    }
-
-    /// Check a few pre-conditions for setting constants, as well as setting
-    /// that `is_const` to true for the particular column reader.
-    fn check_const(&mut self, method_name: &str) {
-        assert!(
-            self.value_decoder.is_none(),
-            "{method_name} cannot be called after set_page_v1/set_page_v2!"
-        );
-        assert!(!self.is_const, "can only set constant once!");
-        self.is_const = true;
-    }
-
     fn check_dictionary(&mut self, encoding: &Encoding) {
         // The column has a dictionary while the new page is of PLAIN encoding. In this case, we
         // should eagerly decode all the dictionary indices and convert the underlying vector to a
diff --git a/native/core/src/parquet/read/mod.rs b/native/core/src/parquet/read/mod.rs
index 5a55f21170..f66f9da3da 100644
--- a/native/core/src/parquet/read/mod.rs
+++ b/native/core/src/parquet/read/mod.rs
@@ -27,9 +27,6 @@ use crate::common::bit::{self, BitReader};
 use arrow::buffer::Buffer;
 use bytes::Buf;
 
-/// Number of bytes to store a decimal value in Arrow value vector
-pub(crate) const DECIMAL_BYTE_WIDTH: usize = 16;
-
 #[derive(Clone, Copy)]
 pub struct ReadOptions {
     // Whether to read legacy dates/timestamps as it is. If false, throw exceptions.
diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
index ce57624b75..60ca2fe3ab 100644
--- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
+++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
@@ -22,7 +22,7 @@ package org.apache.comet.rules
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.util.sideBySide
-import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometScanExec, CometSparkToColumnarExec}
+import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometSparkToColumnarExec}
 import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec}
 import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan}
 import org.apache.spark.sql.execution.adaptive.QueryStageExec
@@ -139,8 +139,7 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa
   private def createColumnarToRowExec(child: SparkPlan): SparkPlan = {
     val schema = child.schema
     val useNative = CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.get() &&
-      CometNativeColumnarToRowExec.supportsSchema(schema) &&
-      !hasScanUsingMutableBuffers(child)
+      CometNativeColumnarToRowExec.supportsSchema(schema)
 
     if (useNative) {
       CometNativeColumnarToRowExec(child)
@@ -149,25 +148,4 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa
     }
   }
 
-  /**
-   * Checks if the plan contains a scan that uses mutable buffers. Native C2R is not compatible
-   * with such scans because the buffers may be modified after C2R reads them.
-   *
-   * This includes:
-   *   - CometScanExec with native_iceberg_compat and partition columns - uses
-   *     ConstantColumnReader
-   */
-  private def hasScanUsingMutableBuffers(op: SparkPlan): Boolean = {
-    op match {
-      case c: QueryStageExec => hasScanUsingMutableBuffers(c.plan)
-      case c: ReusedExchangeExec => hasScanUsingMutableBuffers(c.child)
-      case _ =>
-        op.exists {
-          case scan: CometScanExec =>
-            scan.scanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT &&
-            scan.relation.partitionSchema.nonEmpty
-          case _ => false
-        }
-    }
-  }
 }

From b40b1b0ec1eceedcbca37c203161a12b918ad5e9 Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 16:57:59 -0600
Subject: [PATCH 2/7] fmt

---
 native/core/src/parquet/mod.rs | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)

diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs
index 237c6c7383..11b352eda1 100644
--- a/native/core/src/parquet/mod.rs
+++ b/native/core/src/parquet/mod.rs
@@ -65,9 +65,7 @@ use datafusion::execution::SendableRecordBatchStream;
 use datafusion::physical_plan::ExecutionPlan;
 use datafusion::prelude::{SessionConfig, SessionContext};
 use futures::{poll, StreamExt};
-use jni::objects::{
-    JByteArray, JLongArray, JMap, JObject, JObjectArray, JString, ReleaseMode,
-};
+use jni::objects::{JByteArray, JLongArray, JMap, JObject, JObjectArray, JString, ReleaseMode};
 use jni::sys::{jintArray, JNI_FALSE};
 use object_store::path::Path;
 use read::ColumnReader;

From 64184e5f7f8bdcfa2ee7d534682d051e243fec65 Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 17:53:34 -0600
Subject: [PATCH 3/7] remove arrow_ffi_safe flag

---
 native/core/src/execution/operators/copy.rs   | 29 +++----------------
 native/core/src/execution/operators/mod.rs    |  2 +-
 native/core/src/execution/operators/scan.rs   | 20 +++----------
 native/core/src/execution/planner.rs          | 15 ++--------
 native/proto/src/proto/operator.proto         |  4 +--
 .../operator/CometDataWritingCommand.scala    |  1 -
 .../comet/serde/operator/CometSink.scala      | 14 ---------
 .../comet/CometBroadcastExchangeExec.scala    |  7 -----
 .../sql/comet/CometLocalTableScanExec.scala   |  1 -
 9 files changed, 13 insertions(+), 80 deletions(-)

diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs
index 193c385c3c..4c2ea67578 100644
--- a/native/core/src/execution/operators/copy.rs
+++ b/native/core/src/execution/operators/copy.rs
@@ -22,16 +22,8 @@ use arrow::array::{downcast_dictionary_array, make_array, Array, ArrayRef, Mutab
 use arrow::datatypes::DataType;
 use arrow::error::ArrowError;
 
-#[derive(Debug, PartialEq, Clone)]
-pub enum CopyMode {
-    /// Perform a deep copy and also unpack dictionaries
-    UnpackOrDeepCopy,
-    /// Perform a clone and also unpack dictionaries
-    UnpackOrClone,
-}
-
 /// Copy an Arrow Array
-pub(crate) fn copy_array(array: &dyn Array) -> ArrayRef {
+fn copy_array(array: &dyn Array) -> ArrayRef {
     let capacity = array.len();
     let data = array.to_data();
 
@@ -62,15 +54,8 @@ pub(crate) fn copy_array(array: &dyn Array) -> ArrayRef {
     }
 }
 
-/// Copy an Arrow Array or cast to primitive type if it is a dictionary array.
-/// This is used for `CopyExec` to copy/cast the input array. If the input array
-/// is a dictionary array, we will cast the dictionary array to primitive type
-/// (i.e., unpack the dictionary array) and copy the primitive array. If the input
-/// array is a primitive array, we simply copy the array.
-pub(crate) fn copy_or_unpack_array(
-    array: &Arc,
-    mode: &CopyMode,
-) -> Result {
+/// Unpack dictionary arrays to primitive type, or clone non-dictionary arrays.
+pub(crate) fn copy_or_unpack_array(array: &Arc) -> Result {
     match array.data_type() {
         DataType::Dictionary(_, value_type) => {
             let options = CastOptions::default();
@@ -82,12 +67,6 @@ pub(crate) fn copy_or_unpack_array(
                 &options,
             )?))
         }
-        _ => {
-            if mode == &CopyMode::UnpackOrDeepCopy {
-                Ok(copy_array(array))
-            } else {
-                Ok(Arc::clone(array))
-            }
-        }
+        _ => Ok(Arc::clone(array)),
     }
 }
diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs
index 07ee995367..2780a9edfc 100644
--- a/native/core/src/execution/operators/mod.rs
+++ b/native/core/src/execution/operators/mod.rs
@@ -21,7 +21,7 @@ use std::fmt::Debug;
 
 use jni::objects::GlobalRef;
 
-pub use copy::*;
+pub(crate) use copy::*;
 pub use iceberg_scan::*;
 pub use scan::*;
 
diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs
index 2543705fb0..ef55e20485 100644
--- a/native/core/src/execution/operators/scan.rs
+++ b/native/core/src/execution/operators/scan.rs
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-use crate::execution::operators::{copy_array, copy_or_unpack_array, CopyMode};
+use crate::execution::operators::copy_or_unpack_array;
 use crate::{
     errors::CometError,
     execution::{
@@ -77,8 +77,6 @@ pub struct ScanExec {
     metrics: ExecutionPlanMetricsSet,
     /// Baseline metrics
     baseline_metrics: BaselineMetrics,
-    /// Whether native code can assume ownership of batches that it receives
-    arrow_ffi_safe: bool,
 }
 
 impl ScanExec {
@@ -87,7 +85,6 @@ impl ScanExec {
         input_source: Option>,
         input_source_description: &str,
         data_types: Vec,
-        arrow_ffi_safe: bool,
     ) -> Result {
         let metrics_set = ExecutionPlanMetricsSet::default();
         let baseline_metrics = BaselineMetrics::new(&metrics_set, 0);
@@ -114,7 +111,6 @@ impl ScanExec {
             metrics: metrics_set,
             baseline_metrics,
             schema,
-            arrow_ffi_safe,
         })
     }
 
@@ -147,7 +143,6 @@ impl ScanExec {
                 self.exec_context_id,
                 self.input_source.as_ref().unwrap().as_obj(),
                 self.data_types.len(),
-                self.arrow_ffi_safe,
             )?;
             *current_batch = Some(next_batch);
         }
@@ -162,7 +157,6 @@ impl ScanExec {
         exec_context_id: i64,
         iter: &JObject,
         num_cols: usize,
-        arrow_ffi_safe: bool,
     ) -> Result {
         if exec_context_id == TEST_EXEC_CONTEXT_ID {
             // This is a unit test. We don't need to call JNI.
@@ -225,15 +219,9 @@ impl ScanExec {
                 array
             };
 
-            let array = if arrow_ffi_safe {
-                // ownership of this array has been transferred to native
-                // but we still need to unpack dictionary arrays
-                copy_or_unpack_array(&array, &CopyMode::UnpackOrClone)?
-            } else {
-                // it is necessary to copy the array because the contents may be
-                // overwritten on the JVM side in the future
-                copy_array(&array)
-            };
+            // ownership of this array has been transferred to native
+            // but we still need to unpack dictionary arrays
+            let array = copy_or_unpack_array(&array)?;
 
             inputs.push(array);
 
diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs
index bd37755922..68e426707c 100644
--- a/native/core/src/execution/planner.rs
+++ b/native/core/src/execution/planner.rs
@@ -1266,13 +1266,8 @@ impl PhysicalPlanner {
                     };
 
                 // The `ScanExec` operator will take actual arrays from Spark during execution
-                let scan = ScanExec::new(
-                    self.exec_context_id,
-                    input_source,
-                    &scan.source,
-                    data_types,
-                    scan.arrow_ffi_safe,
-                )?;
+                let scan =
+                    ScanExec::new(self.exec_context_id, input_source, &scan.source, data_types)?;
 
                 Ok((
                     vec![scan.clone()],
@@ -3656,7 +3651,6 @@ mod tests {
                     type_info: None,
                 }],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         };
 
@@ -3721,7 +3715,6 @@ mod tests {
                     type_info: None,
                 }],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         };
 
@@ -3926,7 +3919,6 @@ mod tests {
             op_struct: Some(OpStruct::Scan(spark_operator::Scan {
                 fields: vec![create_proto_datatype()],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         }
     }
@@ -3969,7 +3961,6 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         };
 
@@ -4091,7 +4082,6 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         };
 
@@ -4574,7 +4564,6 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
-                arrow_ffi_safe: false,
             })),
         };
 
diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto
index 4afc1fefb7..3a1a5ca006 100644
--- a/native/proto/src/proto/operator.proto
+++ b/native/proto/src/proto/operator.proto
@@ -81,8 +81,8 @@ message Scan {
   // is purely for informational purposes when viewing native query plans in
   // debug mode.
   string source = 2;
-  // Whether native code can assume ownership of batches that it receives
-  bool arrow_ffi_safe = 3;
+  // field 3 was arrow_ffi_safe (removed - all batches are now ffi safe)
+  reserved 3;
 }
 
 // Common data shared by all partitions in split mode (sent once at planning)
diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
index 69b9bd5f85..4a8ae4d2ac 100644
--- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
@@ -96,7 +96,6 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec
       val scanOp = OperatorOuterClass.Scan
         .newBuilder()
         .setSource(cmd.query.nodeName)
-        .setArrowFfiSafe(false)
 
       // Add fields from the query output schema
       val scanTypes = cmd.query.output.flatMap { attr =>
diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
index ca9dbdad7c..f573fcd061 100644
--- a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
@@ -36,9 +36,6 @@ import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataTy
  */
 abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
 
-  /** Whether the data produced by the Comet operator is FFI safe */
-  def isFfiSafe: Boolean = false
-
   override def enabledConfig: Option[ConfigEntry[Boolean]] = None
 
   override def convert(
@@ -61,8 +58,6 @@ abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
     } else {
       scanBuilder.setSource(source)
     }
-    scanBuilder.setArrowFfiSafe(isFfiSafe)
-
     val scanTypes = op.output.flatten { attr =>
       serializeDataType(attr.dataType)
     }
@@ -86,15 +81,6 @@ abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
 
 object CometExchangeSink extends CometSink[SparkPlan] {
 
-  /**
-   * Exchange data is FFI safe because there is no use of mutable buffers involved.
-   *
-   * Source of broadcast exchange batches is ArrowStreamReader.
-   *
-   * Source of shuffle exchange batches is NativeBatchDecoderIterator.
-   */
-  override def isFfiSafe: Boolean = true
-
   override def createExec(nativeOp: Operator, op: SparkPlan): CometNativeExec =
     CometSinkPlaceHolder(nativeOp, op, op)
 }
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
index 4a323e575a..8012b18b22 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
@@ -279,13 +279,6 @@ case class CometBroadcastExchangeExec(
 
 object CometBroadcastExchangeExec extends CometSink[BroadcastExchangeExec] {
 
-  /**
-   * Exchange data is FFI safe because there is no use of mutable buffers involved.
-   *
-   * Source of broadcast exchange batches is ArrowStreamReader.
-   */
-  override def isFfiSafe: Boolean = true
-
   override def enabledConfig: Option[ConfigEntry[Boolean]] = Some(
     CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED)
 
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
index b804fe347e..5928b76e6e 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
@@ -120,7 +120,6 @@ object CometLocalTableScanExec extends CometOperatorSerde[LocalTableScanExec] {
       .newBuilder()
       .setSource(op.getClass.getSimpleName)
       .addAllFields(scanTypes.asJava)
-      .setArrowFfiSafe(false)
     Some(builder.setScan(scanBuilder).build())
   }
 

From bbdee35e34903af21d68cfe99c8bf29a3095e100 Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 17:59:41 -0600
Subject: [PATCH 4/7] update golden files

---
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    | 10 +++---
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  4 +--
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../q83.native_iceberg_compat/extended.txt    |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    | 12 +++----
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  4 +--
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../extended.txt                              |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    |  8 ++---
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  2 +-
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../q83.native_iceberg_compat/extended.txt    |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  4 +--
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  4 +--
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  2 +-
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 402 files changed, 826 insertions(+), 826 deletions(-)

diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
index 0d0d80c7f3..b7c63811ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
index 63858a8376..0e6397bfc2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
index 5568227543..385e0fdaad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
index 49ad7403ff..466892ba9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometColumnarToRow
+                  :     :     +- CometNativeColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
index c1b791f357..78557e7ffb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
index 55382845fb..b8907128e8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
index 2ad029e444..0f23bf5d4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
index 7958390823..6ae79a435f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
index a8cfde2d31..9c22a76577 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
index 15f5db847d..75511320a4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
index 097f5b20cf..8da34310d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
index 34e673af7c..831d892584 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
index 0d7391c0d8..94af831deb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometColumnarToRow
+            :     :              :     +- CometNativeColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
index 4838389d3f..2cff47fb04 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometColumnarToRow
+      :              :     :           :     +- CometNativeColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
index ea785136aa..d868ceeeb3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
index bbfc91ed08..8b6297f645 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
index 809265b4d5..a778e244c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometColumnarToRow
+:  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometColumnarToRow
+:  :  :  :  :              +- CometNativeColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometColumnarToRow
+:  :  :  :                    +- CometNativeColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometColumnarToRow
+:  :  :                    +- CometNativeColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometColumnarToRow
+:  :                    +- CometNativeColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
index 8c47903ee8..9ca9f72ab4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
index 70bff8abb7..8b2f687037 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
index 780504fb09..3318935f4d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
index c7273327fd..60c49121a6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
index 971bf3c1f6..638a87ab14 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
index 2bf9a90726..e434c5ff76 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
index be3d6196b0..a92eb1903d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
index 715ff39db7..cb4d06350b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
index 2336733a6a..570f9bbcc9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
index 8d2e593975..f9c807168e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
index cb125e2456..1272f205e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
index 7e28f0cb5b..3537e14dcf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
@@ -10,15 +10,15 @@ TakeOrderedAndProject
       :     :     :     +- Filter
       :     :     :        +- Window
       :     :     :           +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :     :              +- CometColumnarToRow
+      :     :     :              +- CometNativeColumnarToRow
       :     :     :                 +- CometSort
       :     :     :                    +- CometColumnarExchange
       :     :     :                       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :     :                          +- CometColumnarToRow
+      :     :     :                          +- CometNativeColumnarToRow
       :     :     :                             +- CometSort
       :     :     :                                +- CometFilter
       :     :     :                                   :  +- Subquery
-      :     :     :                                   :     +- CometColumnarToRow
+      :     :     :                                   :     +- CometNativeColumnarToRow
       :     :     :                                   :        +- CometHashAggregate
       :     :     :                                   :           +- CometExchange
       :     :     :                                   :              +- CometHashAggregate
@@ -36,11 +36,11 @@ TakeOrderedAndProject
       :     :           +- Filter
       :     :              +- Window
       :     :                 +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :                    +- CometColumnarToRow
+      :     :                    +- CometNativeColumnarToRow
       :     :                       +- CometSort
       :     :                          +- CometColumnarExchange
       :     :                             +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :                                +- CometColumnarToRow
+      :     :                                +- CometNativeColumnarToRow
       :     :                                   +- CometSort
       :     :                                      +- CometFilter
       :     :                                         :  +- ReusedSubquery
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
index 23662b8aaf..afdf8d6633 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
index 347702b724..c8325b49ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
index 76c166e7de..4b840c6a27 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
index 958b844c9e..aa0c4c1c38 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
index 8a2badfaff..ea746c5fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometColumnarToRow
+                     :           +- CometNativeColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometColumnarToRow
+                                 +- CometNativeColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
index 0b482769c1..f4318de30f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
index 547d7002b1..013bd790b1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
index 3d50a2131e..3594fb8035 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
index 20115b61c4..e3297b4d66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
index f359e64d7b..783de88195 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
index a5fd7359a3..dc09a1507f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
index 755be00004..b0c27d6da7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
index 7e7f77c2c4..46b1956f4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
index d03ce0e111..d7fde7133a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometColumnarToRow
+                     +- CometNativeColumnarToRow
                         +- CometSort
                            +- CometHashAggregate
                               +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
index 5a472217df..bf624b5ce3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometColumnarToRow
+                  :     :     :  :- CometNativeColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometColumnarToRow
+                  :     :     :     +- CometNativeColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometColumnarToRow
+                  :     :        +- CometNativeColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
index 3aa5ad7551..783b9a64bb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
index 0239787721..95077efbc5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometColumnarToRow
+                                    :- CometNativeColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -37,7 +37,7 @@ TakeOrderedAndProject
                                                    +- Filter
                                                       +- Window
                                                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                            +- CometColumnarToRow
+                                                            +- CometNativeColumnarToRow
                                                                +- CometSort
                                                                   +- CometHashAggregate
                                                                      +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
index 12dc136fee..4de5ab6c3b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
index b56a6590db..685ff36fed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
index a16105c248..0cfc0a8417 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometColumnarToRow
+                  :- CometNativeColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometColumnarToRow
+                  :     :  +- CometNativeColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometColumnarToRow
+                  :     +- CometNativeColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
index 871cf47610..f3e80e01cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
index 4d7a7d1ce7..be5abf4913 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
index dd2aed67bc..ed8d0deb6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
index 731fcea912..eb965c592c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
index 2ab6dd2d0c..17e0a010c0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
index 61e40ee179..fac88c8bc7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
index 5fce933878..fcd8384c22 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
index 5c782b95f8..f99808facc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
index 6444b727e4..c0056e2382 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
index 0c51ae09a5..5fddd74768 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometColumnarToRow
++- CometNativeColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometColumnarToRow
+               :  :- CometNativeColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometColumnarToRow
+               :     +- CometNativeColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
index 1e1247665c..f1e4c8679a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometColumnarToRow
+:  :  :  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
index a86d85c820..e3614131fe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
index aa6c577ed7..3b1bd00423 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
index ff13331764..961299b3d9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
index ca50f78e18..0fb6e6c158 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
index 335f2765d7..79acf0e4eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
index eac4939621..ff3a1a06ad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
index 6ff8eba58f..bcb3e1d444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
index 0f623c9021..0f07a6473f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
index 0969a0e796..4aa20a3755 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
index 152665febf..a18137ef10 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
index 86cf2fc2f3..47e040af97 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
index 0d0d80c7f3..b7c63811ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
index 63858a8376..0e6397bfc2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
index 5568227543..385e0fdaad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
index 49ad7403ff..466892ba9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometColumnarToRow
+                  :     :     +- CometNativeColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
index 49eafd1015..7a4afd3d2b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
index 55382845fb..b8907128e8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
index 2ad029e444..0f23bf5d4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
index 7958390823..6ae79a435f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
index a8cfde2d31..9c22a76577 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
index 15f5db847d..75511320a4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
index 097f5b20cf..8da34310d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
index 34e673af7c..831d892584 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
index 0d7391c0d8..94af831deb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometColumnarToRow
+            :     :              :     +- CometNativeColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
index 4838389d3f..2cff47fb04 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometColumnarToRow
+      :              :     :           :     +- CometNativeColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
index ea785136aa..d868ceeeb3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
index bbfc91ed08..8b6297f645 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
index 809265b4d5..a778e244c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometColumnarToRow
+:  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometColumnarToRow
+:  :  :  :  :              +- CometNativeColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometColumnarToRow
+:  :  :  :                    +- CometNativeColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometColumnarToRow
+:  :  :                    +- CometNativeColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometColumnarToRow
+:  :                    +- CometNativeColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
index 8c47903ee8..9ca9f72ab4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
index 70bff8abb7..8b2f687037 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
index 780504fb09..3318935f4d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
index c7273327fd..60c49121a6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
index 971bf3c1f6..638a87ab14 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
index 2bf9a90726..e434c5ff76 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
index be3d6196b0..a92eb1903d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
index 715ff39db7..cb4d06350b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
index 2336733a6a..570f9bbcc9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
index 8d2e593975..f9c807168e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
index cb125e2456..1272f205e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
index b784a161f7..f7a30e9f90 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
@@ -12,15 +12,15 @@ CometColumnarToRow
          :     :     :        +- Filter
          :     :     :           +- Window
          :     :     :              +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :     :                 +- CometColumnarToRow
+         :     :     :                 +- CometNativeColumnarToRow
          :     :     :                    +- CometSort
          :     :     :                       +- CometColumnarExchange
          :     :     :                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :     :                             +- CometColumnarToRow
+         :     :     :                             +- CometNativeColumnarToRow
          :     :     :                                +- CometSort
          :     :     :                                   +- CometFilter
          :     :     :                                      :  +- Subquery
-         :     :     :                                      :     +- CometColumnarToRow
+         :     :     :                                      :     +- CometNativeColumnarToRow
          :     :     :                                      :        +- CometHashAggregate
          :     :     :                                      :           +- CometExchange
          :     :     :                                      :              +- CometHashAggregate
@@ -39,11 +39,11 @@ CometColumnarToRow
          :     :              +- Filter
          :     :                 +- Window
          :     :                    +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :                       +- CometColumnarToRow
+         :     :                       +- CometNativeColumnarToRow
          :     :                          +- CometSort
          :     :                             +- CometColumnarExchange
          :     :                                +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :                                   +- CometColumnarToRow
+         :     :                                   +- CometNativeColumnarToRow
          :     :                                      +- CometSort
          :     :                                         +- CometFilter
          :     :                                            :  +- ReusedSubquery
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
index 23662b8aaf..afdf8d6633 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
index 347702b724..c8325b49ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
index 76c166e7de..4b840c6a27 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
index 958b844c9e..aa0c4c1c38 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
index 8a2badfaff..ea746c5fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometColumnarToRow
+                     :           +- CometNativeColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometColumnarToRow
+                                 +- CometNativeColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
index 0e8e75c034..7df6105d58 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
index 926061b466..7190b29a9b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
index 3d50a2131e..3594fb8035 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
index 4bb14d3c19..ee55822981 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
index f359e64d7b..783de88195 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
index a5fd7359a3..dc09a1507f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
index 755be00004..b0c27d6da7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
index 7e7f77c2c4..46b1956f4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
index d03ce0e111..d7fde7133a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometColumnarToRow
+                     +- CometNativeColumnarToRow
                         +- CometSort
                            +- CometHashAggregate
                               +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
index 5a472217df..bf624b5ce3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometColumnarToRow
+                  :     :     :  :- CometNativeColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometColumnarToRow
+                  :     :     :     +- CometNativeColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometColumnarToRow
+                  :     :        +- CometNativeColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
index 3aa5ad7551..783b9a64bb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
index 0239787721..95077efbc5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometColumnarToRow
+                                    :- CometNativeColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -37,7 +37,7 @@ TakeOrderedAndProject
                                                    +- Filter
                                                       +- Window
                                                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                            +- CometColumnarToRow
+                                                            +- CometNativeColumnarToRow
                                                                +- CometSort
                                                                   +- CometHashAggregate
                                                                      +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
index 12dc136fee..4de5ab6c3b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
index b56a6590db..685ff36fed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
index a16105c248..0cfc0a8417 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometColumnarToRow
+                  :- CometNativeColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometColumnarToRow
+                  :     :  +- CometNativeColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometColumnarToRow
+                  :     +- CometNativeColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
index 871cf47610..f3e80e01cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
index 4d7a7d1ce7..be5abf4913 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
index dd2aed67bc..ed8d0deb6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
index 731fcea912..eb965c592c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
index 2ab6dd2d0c..17e0a010c0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
index 61e40ee179..fac88c8bc7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
index 5fce933878..fcd8384c22 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
index 5c782b95f8..f99808facc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
index 6444b727e4..c0056e2382 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
index 0c51ae09a5..5fddd74768 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometColumnarToRow
++- CometNativeColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometColumnarToRow
+               :  :- CometNativeColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometColumnarToRow
+               :     +- CometNativeColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
index 1e1247665c..f1e4c8679a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometColumnarToRow
+:  :  :  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
index a86d85c820..e3614131fe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
index aa6c577ed7..3b1bd00423 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
index ff13331764..961299b3d9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
index ca50f78e18..0fb6e6c158 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
index 335f2765d7..79acf0e4eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
index eac4939621..ff3a1a06ad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
index 6ff8eba58f..bcb3e1d444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
index 0f623c9021..0f07a6473f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
index 0969a0e796..4aa20a3755 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
index 152665febf..a18137ef10 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
index 86cf2fc2f3..47e040af97 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
index 0d0d80c7f3..b7c63811ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
index 63858a8376..0e6397bfc2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
index 5568227543..385e0fdaad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
index 49ad7403ff..466892ba9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometColumnarToRow
+                  :     :     +- CometNativeColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
index c1b791f357..78557e7ffb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
index 55382845fb..b8907128e8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
index 2ad029e444..0f23bf5d4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
index 7958390823..6ae79a435f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
index a8cfde2d31..9c22a76577 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
index 15f5db847d..75511320a4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
index 097f5b20cf..8da34310d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
index 34e673af7c..831d892584 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
index 0d7391c0d8..94af831deb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometColumnarToRow
+            :     :              :     +- CometNativeColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
index 4838389d3f..2cff47fb04 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometColumnarToRow
+      :              :     :           :     +- CometNativeColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
index 12050b3ad1..36748b40ea 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometColumnarToRow
+:        +- CometNativeColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometColumnarToRow
+:                                   :- CometNativeColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometColumnarToRow
+                              :- CometNativeColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
index 21afc9cb01..52ca742891 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
index ea785136aa..d868ceeeb3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
index bbfc91ed08..8b6297f645 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
index 809265b4d5..a778e244c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometColumnarToRow
+:  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometColumnarToRow
+:  :  :  :  :              +- CometNativeColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometColumnarToRow
+:  :  :  :                    +- CometNativeColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometColumnarToRow
+:  :  :                    +- CometNativeColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometColumnarToRow
+:  :                    +- CometNativeColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometColumnarToRow
+:                    +- CometNativeColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
index 8c47903ee8..9ca9f72ab4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
index 70bff8abb7..8b2f687037 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
index 780504fb09..3318935f4d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
index c7273327fd..60c49121a6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
index 971bf3c1f6..638a87ab14 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
index 2bf9a90726..e434c5ff76 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
index be3d6196b0..a92eb1903d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
index 715ff39db7..cb4d06350b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
index 8a5a4af707..e8f621d32a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
index 2336733a6a..570f9bbcc9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
index 8d2e593975..f9c807168e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
index cb125e2456..1272f205e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
index 0952fbf8d5..259056f9f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
@@ -9,12 +9,12 @@ TakeOrderedAndProject
       :     :     :  +- Project
       :     :     :     +- Filter
       :     :     :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :     :           +- CometColumnarToRow
+      :     :     :           +- CometNativeColumnarToRow
       :     :     :              +- CometSort
       :     :     :                 +- CometExchange
       :     :     :                    +- CometFilter
       :     :     :                       :  +- Subquery
-      :     :     :                       :     +- CometColumnarToRow
+      :     :     :                       :     +- CometNativeColumnarToRow
       :     :     :                       :        +- CometHashAggregate
       :     :     :                       :           +- CometExchange
       :     :     :                       :              +- CometHashAggregate
@@ -31,12 +31,12 @@ TakeOrderedAndProject
       :     :        +- Project
       :     :           +- Filter
       :     :              +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :                 +- CometColumnarToRow
+      :     :                 +- CometNativeColumnarToRow
       :     :                    +- CometSort
       :     :                       +- CometExchange
       :     :                          +- CometFilter
       :     :                             :  +- Subquery
-      :     :                             :     +- CometColumnarToRow
+      :     :                             :     +- CometNativeColumnarToRow
       :     :                             :        +- CometHashAggregate
       :     :                             :           +- CometExchange
       :     :                             :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
index 23662b8aaf..afdf8d6633 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
index 347702b724..c8325b49ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
index 76c166e7de..4b840c6a27 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
index 958b844c9e..aa0c4c1c38 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
index 8a2badfaff..ea746c5fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometColumnarToRow
+                     :           +- CometNativeColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometColumnarToRow
+                                 +- CometNativeColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
index 0b482769c1..f4318de30f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
index a292badf5d..e5122dc8d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
index e503b00314..81412250ec 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
index 3d50a2131e..3594fb8035 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
index 20115b61c4..e3297b4d66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
index 22c07c4964..faef2d5263 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
index f359e64d7b..783de88195 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
index a5fd7359a3..dc09a1507f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
index 755be00004..b0c27d6da7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
index 7e7f77c2c4..46b1956f4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
index 0ba8f6b0e7..da0bef861d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
index 28f2b6110d..07093ae00b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
index 5a472217df..bf624b5ce3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometColumnarToRow
+                  :     :     :  :- CometNativeColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometColumnarToRow
+                  :     :     :     +- CometNativeColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometColumnarToRow
+                  :     :        +- CometNativeColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
index 3aa5ad7551..783b9a64bb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
index fa1bf8daa4..14e4cb087a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometColumnarToRow
+                                    :- CometNativeColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -36,7 +36,7 @@ TakeOrderedAndProject
                                                 +- Project
                                                    +- Filter
                                                       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                         +- CometColumnarToRow
+                                                         +- CometNativeColumnarToRow
                                                             +- CometSort
                                                                +- CometHashAggregate
                                                                   +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
index 12dc136fee..4de5ab6c3b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
index b56a6590db..685ff36fed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
index a16105c248..0cfc0a8417 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometColumnarToRow
+                  :- CometNativeColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometColumnarToRow
+                  :     :  +- CometNativeColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometColumnarToRow
+                  :     +- CometNativeColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
index 871cf47610..f3e80e01cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
index 4d7a7d1ce7..be5abf4913 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
index dd2aed67bc..ed8d0deb6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
index 731fcea912..eb965c592c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
index 2ab6dd2d0c..17e0a010c0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
index 9ab4ae91f3..3e72e3d553 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
index 5fce933878..fcd8384c22 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
index 5c782b95f8..f99808facc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
index 6444b727e4..c0056e2382 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
index 0c51ae09a5..5fddd74768 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometColumnarToRow
++- CometNativeColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometColumnarToRow
+               :  :- CometNativeColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometColumnarToRow
+               :     +- CometNativeColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
index 1e1247665c..f1e4c8679a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometColumnarToRow
+:  :  :  :  :  :  :- CometNativeColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometColumnarToRow
+:  :  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometColumnarToRow
+:  :  :  :     +- CometNativeColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometColumnarToRow
+:  :  :     +- CometNativeColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometColumnarToRow
+:  :     +- CometNativeColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometColumnarToRow
+:     +- CometNativeColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
index 6b4c3e3383..093505a6cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
index a86d85c820..e3614131fe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometColumnarToRow
+:  :  +- CometNativeColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
index aa6c577ed7..3b1bd00423 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometColumnarToRow
+   :- CometNativeColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
index ff13331764..961299b3d9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
index ca50f78e18..0fb6e6c158 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
index 335f2765d7..79acf0e4eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
index eac4939621..ff3a1a06ad 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
index 6ff8eba58f..bcb3e1d444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
index 0f623c9021..0f07a6473f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
index 0969a0e796..4aa20a3755 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
index 152665febf..a18137ef10 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
index 86cf2fc2f3..47e040af97 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
index c1b791f357..78557e7ffb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
index 3bacf3c8bc..a8adb0443d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometColumnarToRow
+               :           :  :     +- CometNativeColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometColumnarToRow
+                                    :  :     +- CometNativeColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
index 148f6d4a6d..94f7834f85 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
index 5da4848ebe..f642ebc712 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
index 4ccef655b9..d83496133a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
index 91b709aa47..5d8bc6a064 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometColumnarToRow
+         :        +- CometNativeColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometColumnarToRow
+         :                    +- CometNativeColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometColumnarToRow
+         :                                   :- CometNativeColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometColumnarToRow
+                                       :- CometNativeColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
index a3e6b17491..e7288e7e47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
index 968ea0611f..e9571a1f12 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
index 05c8752bd2..6560795de1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometColumnarToRow
+               :                    :        +- CometNativeColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometColumnarToRow
+               :                    :                       :     +- CometNativeColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometColumnarToRow
+               :                    :                                +- CometNativeColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometColumnarToRow
+               :                             +- CometNativeColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometColumnarToRow
+               :                                            :     +- CometNativeColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometColumnarToRow
+               :                                                     +- CometNativeColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometColumnarToRow
+                                          :        +- CometNativeColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometColumnarToRow
+                                          :                       :     +- CometNativeColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometColumnarToRow
+                                          :                                +- CometNativeColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometColumnarToRow
+                                                   +- CometNativeColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometColumnarToRow
+                                                                  :     +- CometNativeColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometColumnarToRow
+                                                                           +- CometNativeColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
index 24167a2372..e2832782b8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
index 20115b61c4..e3297b4d66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
index 407fc9836c..2afcab203a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometColumnarToRow
+                     +- CometNativeColumnarToRow
                         +- CometSort
                            +- CometUnion
                               :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
index 0ab48d6a12..b21df154e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometColumnarToRow
+                           :                 :- CometNativeColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                            :                                +- Filter
                            :                                   +- Window
                            :                                      +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                         +- CometColumnarToRow
+                           :                                         +- CometNativeColumnarToRow
                            :                                            +- CometSort
                            :                                               +- CometHashAggregate
                            :                                                  +- CometExchange
@@ -61,16 +61,16 @@ TakeOrderedAndProject
                            :                                                                    +- CometFilter
                            :                                                                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometColumnarToRow
+                           :              +- CometNativeColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometColumnarToRow
+                           :                             :- CometNativeColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -96,7 +96,7 @@ TakeOrderedAndProject
                            :                                            +- Filter
                            :                                               +- Window
                            :                                                  +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                                     +- CometColumnarToRow
+                           :                                                     +- CometNativeColumnarToRow
                            :                                                        +- CometSort
                            :                                                           +- CometHashAggregate
                            :                                                              +- CometExchange
@@ -117,16 +117,16 @@ TakeOrderedAndProject
                            :                                                                                +- CometFilter
                            :                                                                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometColumnarToRow
+                                                         :- CometNativeColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -152,7 +152,7 @@ TakeOrderedAndProject
                                                                         +- Filter
                                                                            +- Window
                                                                               +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                                                 +- CometColumnarToRow
+                                                                                 +- CometNativeColumnarToRow
                                                                                     +- CometSort
                                                                                        +- CometHashAggregate
                                                                                           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
index 40e2c31863..3578f807aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometColumnarToRow
+               :              :- CometNativeColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometColumnarToRow
+               :              :     :  +- CometNativeColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometColumnarToRow
+               :              :     +- CometNativeColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometColumnarToRow
+               :                          :- CometNativeColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometColumnarToRow
+               :                          :     :  +- CometNativeColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometColumnarToRow
+               :                          :     +- CometNativeColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometColumnarToRow
+               :                          +- CometNativeColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometColumnarToRow
+                                          :- CometNativeColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometColumnarToRow
+                                          :     :  +- CometNativeColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometColumnarToRow
+                                          :     +- CometNativeColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
index 943d6e8d1a..fbd7cba46d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
index 13ff1e34cb..58d00a427d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
index 91e569a85c..1a985ead1b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
index 49eafd1015..7a4afd3d2b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
index 3bacf3c8bc..a8adb0443d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometColumnarToRow
+               :           :  :     +- CometNativeColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometColumnarToRow
+                                    :  :     +- CometNativeColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
index 148f6d4a6d..94f7834f85 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
index 5da4848ebe..f642ebc712 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
index 4ccef655b9..d83496133a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
index 91b709aa47..5d8bc6a064 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometColumnarToRow
+         :        +- CometNativeColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometColumnarToRow
+         :                    +- CometNativeColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometColumnarToRow
+         :                                   :- CometNativeColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometColumnarToRow
+                                       :- CometNativeColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
index a3e6b17491..e7288e7e47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
index 968ea0611f..e9571a1f12 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
index 05c8752bd2..6560795de1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometColumnarToRow
+               :                    :        +- CometNativeColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometColumnarToRow
+               :                    :                       :     +- CometNativeColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometColumnarToRow
+               :                    :                                +- CometNativeColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometColumnarToRow
+               :                             +- CometNativeColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometColumnarToRow
+               :                                            :     +- CometNativeColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometColumnarToRow
+               :                                                     +- CometNativeColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometColumnarToRow
+                                          :        +- CometNativeColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometColumnarToRow
+                                          :                       :     +- CometNativeColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometColumnarToRow
+                                          :                                +- CometNativeColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometColumnarToRow
+                                                   +- CometNativeColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometColumnarToRow
+                                                                  :     +- CometNativeColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometColumnarToRow
+                                                                           +- CometNativeColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
index 24167a2372..e2832782b8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
index 4bb14d3c19..ee55822981 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
index 407fc9836c..2afcab203a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometColumnarToRow
+         +- CometNativeColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometColumnarToRow
+                     +- CometNativeColumnarToRow
                         +- CometSort
                            +- CometUnion
                               :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
index 0ab48d6a12..b21df154e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometColumnarToRow
+                           :                 :- CometNativeColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                            :                                +- Filter
                            :                                   +- Window
                            :                                      +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                         +- CometColumnarToRow
+                           :                                         +- CometNativeColumnarToRow
                            :                                            +- CometSort
                            :                                               +- CometHashAggregate
                            :                                                  +- CometExchange
@@ -61,16 +61,16 @@ TakeOrderedAndProject
                            :                                                                    +- CometFilter
                            :                                                                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometColumnarToRow
+                           :              +- CometNativeColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometColumnarToRow
+                           :                             :- CometNativeColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -96,7 +96,7 @@ TakeOrderedAndProject
                            :                                            +- Filter
                            :                                               +- Window
                            :                                                  +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                                     +- CometColumnarToRow
+                           :                                                     +- CometNativeColumnarToRow
                            :                                                        +- CometSort
                            :                                                           +- CometHashAggregate
                            :                                                              +- CometExchange
@@ -117,16 +117,16 @@ TakeOrderedAndProject
                            :                                                                                +- CometFilter
                            :                                                                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometColumnarToRow
+                                                         :- CometNativeColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -152,7 +152,7 @@ TakeOrderedAndProject
                                                                         +- Filter
                                                                            +- Window
                                                                               +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                                                 +- CometColumnarToRow
+                                                                                 +- CometNativeColumnarToRow
                                                                                     +- CometSort
                                                                                        +- CometHashAggregate
                                                                                           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
index 40e2c31863..3578f807aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometColumnarToRow
+               :              :- CometNativeColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometColumnarToRow
+               :              :     :  +- CometNativeColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometColumnarToRow
+               :              :     +- CometNativeColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometColumnarToRow
+               :                          :- CometNativeColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometColumnarToRow
+               :                          :     :  +- CometNativeColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometColumnarToRow
+               :                          :     +- CometNativeColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometColumnarToRow
+               :                          +- CometNativeColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometColumnarToRow
+                                          :- CometNativeColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometColumnarToRow
+                                          :     :  +- CometNativeColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometColumnarToRow
+                                          :     +- CometNativeColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
index 943d6e8d1a..fbd7cba46d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
index 13ff1e34cb..58d00a427d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
index 91e569a85c..1a985ead1b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
index 1812b9e872..eb85a80eb1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
index c1b791f357..78557e7ffb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometColumnarToRow
+      :  :     +- CometNativeColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
index 3bacf3c8bc..a8adb0443d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometColumnarToRow
+               :           :  :     +- CometNativeColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometColumnarToRow
+               :                    :  :     +- CometNativeColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometColumnarToRow
+                                    :  :     +- CometNativeColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
index 148f6d4a6d..94f7834f85 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
index 0cfd81f36f..643e47b0c6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
index 5da4848ebe..f642ebc712 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometColumnarToRow
+                     :- CometNativeColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
index 4ccef655b9..d83496133a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
index 91b709aa47..5d8bc6a064 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometColumnarToRow
+         :        +- CometNativeColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometColumnarToRow
+         :                    +- CometNativeColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometColumnarToRow
+         :                                   :- CometNativeColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometColumnarToRow
+                                       :- CometNativeColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
index a3e6b17491..e7288e7e47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
index 65e639c1d8..18ed3e6335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
index 855ca74e0b..c5125b1861 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometColumnarToRow
+                  :     :        :  :- CometNativeColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometColumnarToRow
+                  :     :        :     +- CometNativeColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometColumnarToRow
+                  :     :           +- CometNativeColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
index fc330dd8b6..bdbfb0d6dc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
index 968ea0611f..e9571a1f12 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
index f38ceaf317..9691120d2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
index d1b415dfe4..0b4672eabb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
index 05c8752bd2..6560795de1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometColumnarToRow
+               :                    :        +- CometNativeColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometColumnarToRow
+               :                    :                       :     +- CometNativeColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometColumnarToRow
+               :                    :                                +- CometNativeColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometColumnarToRow
+               :                             +- CometNativeColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometColumnarToRow
+               :                                            :     +- CometNativeColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometColumnarToRow
+               :                                                     +- CometNativeColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometColumnarToRow
+                        +- CometNativeColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometColumnarToRow
+                                          :        +- CometNativeColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometColumnarToRow
+                                          :                       :     +- CometNativeColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometColumnarToRow
+                                          :                                +- CometNativeColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometColumnarToRow
+                                                   +- CometNativeColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometColumnarToRow
+                                                                  :     +- CometNativeColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometColumnarToRow
+                                                                           +- CometNativeColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
index 1c37fd4e8c..5e8acff889 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometColumnarToRow
+      :     :              +- CometNativeColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometColumnarToRow
+      :              +- CometNativeColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometColumnarToRow
+               +- CometNativeColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
index 24167a2372..e2832782b8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
index 20115b61c4..e3297b4d66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
index 75884c3f9d..a868670a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
index c5c2f024b9..b610537f40 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometUnion
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
index 94b23c3b4f..a75ecb0e47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometColumnarToRow
+                           :                 :- CometNativeColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                            :                             +- Project
                            :                                +- Filter
                            :                                   +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                           :                                      +- CometColumnarToRow
+                           :                                      +- CometNativeColumnarToRow
                            :                                         +- CometSort
                            :                                            +- CometHashAggregate
                            :                                               +- CometExchange
@@ -60,16 +60,16 @@ TakeOrderedAndProject
                            :                                                                 +- CometFilter
                            :                                                                    +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometColumnarToRow
+                           :  +- CometNativeColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometColumnarToRow
+                           :              +- CometNativeColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometColumnarToRow
+                           :                             :- CometNativeColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -94,7 +94,7 @@ TakeOrderedAndProject
                            :                                         +- Project
                            :                                            +- Filter
                            :                                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                           :                                                  +- CometColumnarToRow
+                           :                                                  +- CometNativeColumnarToRow
                            :                                                     +- CometSort
                            :                                                        +- CometHashAggregate
                            :                                                           +- CometExchange
@@ -115,16 +115,16 @@ TakeOrderedAndProject
                            :                                                                             +- CometFilter
                            :                                                                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometColumnarToRow
+                                                         :- CometNativeColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                                                      +- Project
                                                                         +- Filter
                                                                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                              +- CometColumnarToRow
+                                                                              +- CometNativeColumnarToRow
                                                                                  +- CometSort
                                                                                     +- CometHashAggregate
                                                                                        +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
index 819698e99a..22fde3b02c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
index 9fb1ccfbe0..922864baa1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
index a3bbeab3af..ec53af4809 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
index 40e2c31863..3578f807aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometColumnarToRow
+               :              :- CometNativeColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometColumnarToRow
+               :              :     :  +- CometNativeColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometColumnarToRow
+               :              :     +- CometNativeColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometColumnarToRow
+               :  +- CometNativeColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometColumnarToRow
+               :              +- CometNativeColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometColumnarToRow
+               :                          :- CometNativeColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometColumnarToRow
+               :                          :     :  +- CometNativeColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometColumnarToRow
+               :                          :     +- CometNativeColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometColumnarToRow
+               :                          +- CometNativeColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometColumnarToRow
+                  +- CometNativeColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometColumnarToRow
+                              +- CometNativeColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometColumnarToRow
+                                          :- CometNativeColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometColumnarToRow
+                                          :     :  +- CometNativeColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometColumnarToRow
+                                          :     +- CometNativeColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometColumnarToRow
+                                          +- CometNativeColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
index 3b721ab107..1b1e6d0cde 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometColumnarToRow
+   +- CometNativeColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
index 943d6e8d1a..fbd7cba46d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
index 13ff1e34cb..58d00a427d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometColumnarToRow
+      +- CometNativeColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
index 91e569a85c..1a985ead1b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometColumnarToRow
+CometNativeColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometColumnarToRow
+            +- CometNativeColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate

From 48abb031e4a90d77677a404c1cbc103aa3ee65d2 Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 18:32:48 -0600
Subject: [PATCH 5/7] Revert "update golden files"

This reverts commit bbdee35e34903af21d68cfe99c8bf29a3095e100.
---
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    | 10 +++---
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  4 +--
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../q83.native_iceberg_compat/extended.txt    |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    | 12 +++----
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  4 +--
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../extended.txt                              |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q1.native_iceberg_compat/extended.txt     |  2 +-
 .../q10.native_iceberg_compat/extended.txt    |  8 ++---
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q13.native_iceberg_compat/extended.txt    |  2 +-
 .../q14a.native_iceberg_compat/extended.txt   |  4 +--
 .../q14b.native_iceberg_compat/extended.txt   |  4 +--
 .../q15.native_iceberg_compat/extended.txt    |  2 +-
 .../q16.native_iceberg_compat/extended.txt    |  4 +--
 .../q17.native_iceberg_compat/extended.txt    |  2 +-
 .../q18.native_iceberg_compat/extended.txt    |  2 +-
 .../q19.native_iceberg_compat/extended.txt    |  2 +-
 .../q2.native_iceberg_compat/extended.txt     |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q21.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  2 +-
 .../q23a.native_iceberg_compat/extended.txt   |  4 +--
 .../q23b.native_iceberg_compat/extended.txt   |  4 +--
 .../q24a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q24b.native_iceberg_compat/extended.txt   | 12 +++----
 .../q25.native_iceberg_compat/extended.txt    |  2 +-
 .../q26.native_iceberg_compat/extended.txt    |  2 +-
 .../q27.native_iceberg_compat/extended.txt    |  2 +-
 .../q28.native_iceberg_compat/extended.txt    | 24 ++++++-------
 .../q29.native_iceberg_compat/extended.txt    |  2 +-
 .../q3.native_iceberg_compat/extended.txt     |  2 +-
 .../q30.native_iceberg_compat/extended.txt    |  2 +-
 .../q31.native_iceberg_compat/extended.txt    |  2 +-
 .../q32.native_iceberg_compat/extended.txt    |  2 +-
 .../q33.native_iceberg_compat/extended.txt    |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q36.native_iceberg_compat/extended.txt    |  2 +-
 .../q37.native_iceberg_compat/extended.txt    |  2 +-
 .../q38.native_iceberg_compat/extended.txt    |  2 +-
 .../q39a.native_iceberg_compat/extended.txt   |  2 +-
 .../q39b.native_iceberg_compat/extended.txt   |  2 +-
 .../q4.native_iceberg_compat/extended.txt     |  2 +-
 .../q40.native_iceberg_compat/extended.txt    |  2 +-
 .../q42.native_iceberg_compat/extended.txt    |  2 +-
 .../q43.native_iceberg_compat/extended.txt    |  2 +-
 .../q44.native_iceberg_compat/extended.txt    |  8 ++---
 .../q45.native_iceberg_compat/extended.txt    |  4 +--
 .../q46.native_iceberg_compat/extended.txt    |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q48.native_iceberg_compat/extended.txt    |  2 +-
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q5.native_iceberg_compat/extended.txt     |  2 +-
 .../q50.native_iceberg_compat/extended.txt    |  2 +-
 .../q51.native_iceberg_compat/extended.txt    |  6 ++--
 .../q52.native_iceberg_compat/extended.txt    |  2 +-
 .../q53.native_iceberg_compat/extended.txt    |  2 +-
 .../q54.native_iceberg_compat/extended.txt    |  2 +-
 .../q55.native_iceberg_compat/extended.txt    |  2 +-
 .../q56.native_iceberg_compat/extended.txt    |  2 +-
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q58.native_iceberg_compat/extended.txt    |  2 +-
 .../q59.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q60.native_iceberg_compat/extended.txt    |  2 +-
 .../q61.native_iceberg_compat/extended.txt    |  4 +--
 .../q62.native_iceberg_compat/extended.txt    |  2 +-
 .../q63.native_iceberg_compat/extended.txt    |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q65.native_iceberg_compat/extended.txt    |  2 +-
 .../q66.native_iceberg_compat/extended.txt    |  2 +-
 .../q67.native_iceberg_compat/extended.txt    |  2 +-
 .../q68.native_iceberg_compat/extended.txt    |  2 +-
 .../q69.native_iceberg_compat/extended.txt    |  8 ++---
 .../q7.native_iceberg_compat/extended.txt     |  2 +-
 .../q70.native_iceberg_compat/extended.txt    |  8 ++---
 .../q71.native_iceberg_compat/extended.txt    |  2 +-
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q73.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q76.native_iceberg_compat/extended.txt    |  2 +-
 .../q77.native_iceberg_compat/extended.txt    | 10 +++---
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q79.native_iceberg_compat/extended.txt    |  2 +-
 .../q8.native_iceberg_compat/extended.txt     |  2 +-
 .../q80.native_iceberg_compat/extended.txt    |  2 +-
 .../q81.native_iceberg_compat/extended.txt    |  2 +-
 .../q82.native_iceberg_compat/extended.txt    |  2 +-
 .../q83.native_iceberg_compat/extended.txt    |  2 +-
 .../q84.native_iceberg_compat/extended.txt    |  2 +-
 .../q85.native_iceberg_compat/extended.txt    |  2 +-
 .../q86.native_iceberg_compat/extended.txt    |  2 +-
 .../q87.native_iceberg_compat/extended.txt    |  8 ++---
 .../q88.native_iceberg_compat/extended.txt    | 16 ++++-----
 .../q89.native_iceberg_compat/extended.txt    |  2 +-
 .../q9.native_iceberg_compat/extended.txt     | 10 +++---
 .../q90.native_iceberg_compat/extended.txt    |  4 +--
 .../q91.native_iceberg_compat/extended.txt    |  2 +-
 .../q92.native_iceberg_compat/extended.txt    |  2 +-
 .../q93.native_iceberg_compat/extended.txt    |  2 +-
 .../q94.native_iceberg_compat/extended.txt    |  4 +--
 .../q95.native_iceberg_compat/extended.txt    |  4 +--
 .../q96.native_iceberg_compat/extended.txt    |  2 +-
 .../q97.native_iceberg_compat/extended.txt    |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q99.native_iceberg_compat/extended.txt    |  2 +-
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  4 +--
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  4 +--
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 .../q10a.native_iceberg_compat/extended.txt   |  2 +-
 .../q11.native_iceberg_compat/extended.txt    |  2 +-
 .../q12.native_iceberg_compat/extended.txt    |  2 +-
 .../q14.native_iceberg_compat/extended.txt    |  4 +--
 .../q14a.native_iceberg_compat/extended.txt   | 12 +++----
 .../q18a.native_iceberg_compat/extended.txt   |  2 +-
 .../q20.native_iceberg_compat/extended.txt    |  2 +-
 .../q22.native_iceberg_compat/extended.txt    |  4 +--
 .../q22a.native_iceberg_compat/extended.txt   |  2 +-
 .../q24.native_iceberg_compat/extended.txt    | 14 ++++----
 .../q27a.native_iceberg_compat/extended.txt   |  2 +-
 .../q34.native_iceberg_compat/extended.txt    |  2 +-
 .../q35.native_iceberg_compat/extended.txt    |  8 ++---
 .../q35a.native_iceberg_compat/extended.txt   |  2 +-
 .../q36a.native_iceberg_compat/extended.txt   |  2 +-
 .../q47.native_iceberg_compat/extended.txt    |  6 ++--
 .../q49.native_iceberg_compat/extended.txt    |  8 ++---
 .../q51a.native_iceberg_compat/extended.txt   | 28 +++++++--------
 .../q57.native_iceberg_compat/extended.txt    |  6 ++--
 .../q5a.native_iceberg_compat/extended.txt    |  2 +-
 .../q6.native_iceberg_compat/extended.txt     |  2 +-
 .../q64.native_iceberg_compat/extended.txt    |  2 +-
 .../q67a.native_iceberg_compat/extended.txt   |  2 +-
 .../q70a.native_iceberg_compat/extended.txt   | 24 ++++++-------
 .../q72.native_iceberg_compat/extended.txt    |  2 +-
 .../q74.native_iceberg_compat/extended.txt    |  2 +-
 .../q75.native_iceberg_compat/extended.txt    |  2 +-
 .../q77a.native_iceberg_compat/extended.txt   | 36 +++++++++----------
 .../q78.native_iceberg_compat/extended.txt    |  2 +-
 .../q80a.native_iceberg_compat/extended.txt   |  2 +-
 .../q86a.native_iceberg_compat/extended.txt   |  2 +-
 .../q98.native_iceberg_compat/extended.txt    |  4 +--
 402 files changed, 826 insertions(+), 826 deletions(-)

diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
index b7c63811ca..0d0d80c7f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
index 0e6397bfc2..63858a8376 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
index 385e0fdaad..5568227543 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
index 466892ba9d..49ad7403ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometNativeColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometNativeColumnarToRow
+                  :     :     +- CometColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
index 78557e7ffb..c1b791f357 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
index b8907128e8..55382845fb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
index 0f23bf5d4e..2ad029e444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
index 6ae79a435f..7958390823 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
index 9c22a76577..a8cfde2d31 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
index 75511320a4..15f5db847d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
index 8da34310d7..097f5b20cf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
index 831d892584..34e673af7c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
index 94af831deb..0d7391c0d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometNativeColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometNativeColumnarToRow
+            :     :              :     +- CometColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
index 2cff47fb04..4838389d3f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometNativeColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometNativeColumnarToRow
+      :              :     :           :     +- CometColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
index d868ceeeb3..ea785136aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
index 8b6297f645..bbfc91ed08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
index a778e244c6..809265b4d5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometNativeColumnarToRow
+:  :  :  :  :              +- CometColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometNativeColumnarToRow
+:  :  :  :                    +- CometColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometNativeColumnarToRow
+:  :  :                    +- CometColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometNativeColumnarToRow
+:  :                    +- CometColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
index 9ca9f72ab4..8c47903ee8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
index 8b2f687037..70bff8abb7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
index 3318935f4d..780504fb09 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
index 60c49121a6..c7273327fd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
index 638a87ab14..971bf3c1f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
index e434c5ff76..2bf9a90726 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
index a92eb1903d..be3d6196b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
index cb4d06350b..715ff39db7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
index 570f9bbcc9..2336733a6a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
index f9c807168e..8d2e593975 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
index 1272f205e4..cb125e2456 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
index 3537e14dcf..7e28f0cb5b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt
@@ -10,15 +10,15 @@ TakeOrderedAndProject
       :     :     :     +- Filter
       :     :     :        +- Window
       :     :     :           +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :     :              +- CometNativeColumnarToRow
+      :     :     :              +- CometColumnarToRow
       :     :     :                 +- CometSort
       :     :     :                    +- CometColumnarExchange
       :     :     :                       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :     :                          +- CometNativeColumnarToRow
+      :     :     :                          +- CometColumnarToRow
       :     :     :                             +- CometSort
       :     :     :                                +- CometFilter
       :     :     :                                   :  +- Subquery
-      :     :     :                                   :     +- CometNativeColumnarToRow
+      :     :     :                                   :     +- CometColumnarToRow
       :     :     :                                   :        +- CometHashAggregate
       :     :     :                                   :           +- CometExchange
       :     :     :                                   :              +- CometHashAggregate
@@ -36,11 +36,11 @@ TakeOrderedAndProject
       :     :           +- Filter
       :     :              +- Window
       :     :                 +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :                    +- CometNativeColumnarToRow
+      :     :                    +- CometColumnarToRow
       :     :                       +- CometSort
       :     :                          +- CometColumnarExchange
       :     :                             +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-      :     :                                +- CometNativeColumnarToRow
+      :     :                                +- CometColumnarToRow
       :     :                                   +- CometSort
       :     :                                      +- CometFilter
       :     :                                         :  +- ReusedSubquery
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
index afdf8d6633..23662b8aaf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
index c8325b49ca..347702b724 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
index 4b840c6a27..76c166e7de 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
index aa0c4c1c38..958b844c9e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
index ea746c5fee..8a2badfaff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometNativeColumnarToRow
+                     :           +- CometColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometNativeColumnarToRow
+                                 +- CometColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
index f4318de30f..0b482769c1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
index 013bd790b1..547d7002b1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
index 3594fb8035..3d50a2131e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
index e3297b4d66..20115b61c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
index 783de88195..f359e64d7b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
index dc09a1507f..a5fd7359a3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
index b0c27d6da7..755be00004 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
index 46b1956f4e..7e7f77c2c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
index d7fde7133a..d03ce0e111 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometNativeColumnarToRow
+                     +- CometColumnarToRow
                         +- CometSort
                            +- CometHashAggregate
                               +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
index bf624b5ce3..5a472217df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometNativeColumnarToRow
+                  :     :     :  :- CometColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometNativeColumnarToRow
+                  :     :     :     +- CometColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometNativeColumnarToRow
+                  :     :        +- CometColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
index 783b9a64bb..3aa5ad7551 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
index 95077efbc5..0239787721 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometNativeColumnarToRow
+                                    :- CometColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -37,7 +37,7 @@ TakeOrderedAndProject
                                                    +- Filter
                                                       +- Window
                                                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                            +- CometNativeColumnarToRow
+                                                            +- CometColumnarToRow
                                                                +- CometSort
                                                                   +- CometHashAggregate
                                                                      +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
index 4de5ab6c3b..12dc136fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
index 685ff36fed..b56a6590db 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
index 0cfc0a8417..a16105c248 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometNativeColumnarToRow
+                  :- CometColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometNativeColumnarToRow
+                  :     :  +- CometColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometNativeColumnarToRow
+                  :     +- CometColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
index f3e80e01cb..871cf47610 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
index be5abf4913..4d7a7d1ce7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
index ed8d0deb6d..dd2aed67bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
index eb965c592c..731fcea912 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
index 17e0a010c0..2ab6dd2d0c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
index fac88c8bc7..61e40ee179 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
index fcd8384c22..5fce933878 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
index f99808facc..5c782b95f8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
index c0056e2382..6444b727e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
index 5fddd74768..0c51ae09a5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometNativeColumnarToRow
++- CometColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometNativeColumnarToRow
+               :  :- CometColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometNativeColumnarToRow
+               :     +- CometColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
index f1e4c8679a..1e1247665c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
index e3614131fe..a86d85c820 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
index 3b1bd00423..aa6c577ed7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
index 961299b3d9..ff13331764 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
index 0fb6e6c158..ca50f78e18 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
index 79acf0e4eb..335f2765d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
index ff3a1a06ad..eac4939621 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
index bcb3e1d444..6ff8eba58f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
index 0f07a6473f..0f623c9021 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
index 4aa20a3755..0969a0e796 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
index a18137ef10..152665febf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
index 47e040af97..86cf2fc2f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
index b7c63811ca..0d0d80c7f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
index 0e6397bfc2..63858a8376 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
index 385e0fdaad..5568227543 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
index 466892ba9d..49ad7403ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometNativeColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometNativeColumnarToRow
+                  :     :     +- CometColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
index 7a4afd3d2b..49eafd1015 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
index b8907128e8..55382845fb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
index 0f23bf5d4e..2ad029e444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
index 6ae79a435f..7958390823 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
index 9c22a76577..a8cfde2d31 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
index 75511320a4..15f5db847d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
index 8da34310d7..097f5b20cf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
index 831d892584..34e673af7c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
index 94af831deb..0d7391c0d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometNativeColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometNativeColumnarToRow
+            :     :              :     +- CometColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
index 2cff47fb04..4838389d3f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometNativeColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometNativeColumnarToRow
+      :              :     :           :     +- CometColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
index d868ceeeb3..ea785136aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
index 8b6297f645..bbfc91ed08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
index a778e244c6..809265b4d5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometNativeColumnarToRow
+:  :  :  :  :              +- CometColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometNativeColumnarToRow
+:  :  :  :                    +- CometColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometNativeColumnarToRow
+:  :  :                    +- CometColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometNativeColumnarToRow
+:  :                    +- CometColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
index 9ca9f72ab4..8c47903ee8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
index 8b2f687037..70bff8abb7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
index 3318935f4d..780504fb09 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
index 60c49121a6..c7273327fd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
index 638a87ab14..971bf3c1f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
index e434c5ff76..2bf9a90726 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
index a92eb1903d..be3d6196b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
index cb4d06350b..715ff39db7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
index 570f9bbcc9..2336733a6a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
index f9c807168e..8d2e593975 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
index 1272f205e4..cb125e2456 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
index f7a30e9f90..b784a161f7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
@@ -12,15 +12,15 @@ CometNativeColumnarToRow
          :     :     :        +- Filter
          :     :     :           +- Window
          :     :     :              +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :     :                 +- CometNativeColumnarToRow
+         :     :     :                 +- CometColumnarToRow
          :     :     :                    +- CometSort
          :     :     :                       +- CometColumnarExchange
          :     :     :                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :     :                             +- CometNativeColumnarToRow
+         :     :     :                             +- CometColumnarToRow
          :     :     :                                +- CometSort
          :     :     :                                   +- CometFilter
          :     :     :                                      :  +- Subquery
-         :     :     :                                      :     +- CometNativeColumnarToRow
+         :     :     :                                      :     +- CometColumnarToRow
          :     :     :                                      :        +- CometHashAggregate
          :     :     :                                      :           +- CometExchange
          :     :     :                                      :              +- CometHashAggregate
@@ -39,11 +39,11 @@ CometNativeColumnarToRow
          :     :              +- Filter
          :     :                 +- Window
          :     :                    +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :                       +- CometNativeColumnarToRow
+         :     :                       +- CometColumnarToRow
          :     :                          +- CometSort
          :     :                             +- CometColumnarExchange
          :     :                                +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         :     :                                   +- CometNativeColumnarToRow
+         :     :                                   +- CometColumnarToRow
          :     :                                      +- CometSort
          :     :                                         +- CometFilter
          :     :                                            :  +- ReusedSubquery
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
index afdf8d6633..23662b8aaf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
index c8325b49ca..347702b724 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
index 4b840c6a27..76c166e7de 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
index aa0c4c1c38..958b844c9e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
index ea746c5fee..8a2badfaff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometNativeColumnarToRow
+                     :           +- CometColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometNativeColumnarToRow
+                                 +- CometColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
index 7df6105d58..0e8e75c034 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
index 7190b29a9b..926061b466 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
index 3594fb8035..3d50a2131e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
index ee55822981..4bb14d3c19 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
index 783de88195..f359e64d7b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
index dc09a1507f..a5fd7359a3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
index b0c27d6da7..755be00004 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
index 46b1956f4e..7e7f77c2c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
index d7fde7133a..d03ce0e111 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometNativeColumnarToRow
+                     +- CometColumnarToRow
                         +- CometSort
                            +- CometHashAggregate
                               +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
index bf624b5ce3..5a472217df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometNativeColumnarToRow
+                  :     :     :  :- CometColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometNativeColumnarToRow
+                  :     :     :     +- CometColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometNativeColumnarToRow
+                  :     :        +- CometColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
index 783b9a64bb..3aa5ad7551 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
index 95077efbc5..0239787721 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometNativeColumnarToRow
+                                    :- CometColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -37,7 +37,7 @@ TakeOrderedAndProject
                                                    +- Filter
                                                       +- Window
                                                          +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                            +- CometNativeColumnarToRow
+                                                            +- CometColumnarToRow
                                                                +- CometSort
                                                                   +- CometHashAggregate
                                                                      +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
index 4de5ab6c3b..12dc136fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
index 685ff36fed..b56a6590db 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
index 0cfc0a8417..a16105c248 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometNativeColumnarToRow
+                  :- CometColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometNativeColumnarToRow
+                  :     :  +- CometColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometNativeColumnarToRow
+                  :     +- CometColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
index f3e80e01cb..871cf47610 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
index be5abf4913..4d7a7d1ce7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
index ed8d0deb6d..dd2aed67bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
index eb965c592c..731fcea912 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
index 17e0a010c0..2ab6dd2d0c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
index fac88c8bc7..61e40ee179 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
index fcd8384c22..5fce933878 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
index f99808facc..5c782b95f8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
index c0056e2382..6444b727e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
index 5fddd74768..0c51ae09a5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometNativeColumnarToRow
++- CometColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometNativeColumnarToRow
+               :  :- CometColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometNativeColumnarToRow
+               :     +- CometColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
index f1e4c8679a..1e1247665c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
index e3614131fe..a86d85c820 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
index 3b1bd00423..aa6c577ed7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
index 961299b3d9..ff13331764 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
index 0fb6e6c158..ca50f78e18 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
index 79acf0e4eb..335f2765d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
index ff3a1a06ad..eac4939621 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
index bcb3e1d444..6ff8eba58f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
index 0f07a6473f..0f623c9021 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
index 4aa20a3755..0969a0e796 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
index a18137ef10..152665febf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
index 47e040af97..86cf2fc2f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
index b7c63811ca..0d0d80c7f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
index 0e6397bfc2..63858a8376 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
index 385e0fdaad..5568227543 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
index 466892ba9d..49ad7403ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -8,7 +8,7 @@ CometNativeColumnarToRow
                   :- CometProject
                   :  +- CometFilter
                   :     :  +- Subquery
-                  :     :     +- CometNativeColumnarToRow
+                  :     :     +- CometColumnarToRow
                   :     :        +- CometHashAggregate
                   :     :           +- CometExchange
                   :     :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
index 78557e7ffb..c1b791f357 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
index b8907128e8..55382845fb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
index 0f23bf5d4e..2ad029e444 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
index 6ae79a435f..7958390823 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
index 9c22a76577..a8cfde2d31 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
index 75511320a4..15f5db847d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
index 8da34310d7..097f5b20cf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
index 831d892584..34e673af7c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
index 94af831deb..0d7391c0d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
@@ -47,7 +47,7 @@ CometNativeColumnarToRow
             :     :        +- CometProject
             :     :           +- CometFilter
             :     :              :  +- Subquery
-            :     :              :     +- CometNativeColumnarToRow
+            :     :              :     +- CometColumnarToRow
             :     :              :        +- CometHashAggregate
             :     :              :           +- CometExchange
             :     :              :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
index 2cff47fb04..4838389d3f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
@@ -50,7 +50,7 @@ CometNativeColumnarToRow
       :              :     :     +- CometProject
       :              :     :        +- CometFilter
       :              :     :           :  +- Subquery
-      :              :     :           :     +- CometNativeColumnarToRow
+      :              :     :           :     +- CometColumnarToRow
       :              :     :           :        +- CometHashAggregate
       :              :     :           :           +- CometExchange
       :              :     :           :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
index 36748b40ea..12050b3ad1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt
@@ -1,16 +1,16 @@
 Filter
 :  +- Subquery
 :     +- HashAggregate
-:        +- CometNativeColumnarToRow
+:        +- CometColumnarToRow
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +- HashAggregate
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometColumnarExchange
 :                          +- HashAggregate
 :                             +- Project
 :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-:                                   :- CometNativeColumnarToRow
+:                                   :- CometColumnarToRow
 :                                   :  +- CometProject
 :                                   :     +- CometBroadcastHashJoin
 :                                   :        :- CometProject
@@ -47,16 +47,16 @@ Filter
 :                                            +- CometFilter
 :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- HashAggregate
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometColumnarExchange
                      +- HashAggregate
                         +- Project
                            +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                              :- CometNativeColumnarToRow
+                              :- CometColumnarToRow
                               :  +- CometProject
                               :     +- CometBroadcastHashJoin
                               :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
index 52ca742891..21afc9cb01 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
index d868ceeeb3..ea785136aa 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
index 8b6297f645..bbfc91ed08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
index a778e244c6..809265b4d5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt
@@ -3,72 +3,72 @@ BroadcastNestedLoopJoin
 :  :- BroadcastNestedLoopJoin
 :  :  :- BroadcastNestedLoopJoin
 :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :     +- CometColumnarExchange
 :  :  :  :  :        +- HashAggregate
 :  :  :  :  :           +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :  :              +- CometNativeColumnarToRow
+:  :  :  :  :              +- CometColumnarToRow
 :  :  :  :  :                 +- CometExchange
 :  :  :  :  :                    +- CometHashAggregate
 :  :  :  :  :                       +- CometProject
 :  :  :  :  :                          +- CometFilter
 :  :  :  :  :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometColumnarExchange
 :  :  :  :              +- HashAggregate
 :  :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :  :                    +- CometNativeColumnarToRow
+:  :  :  :                    +- CometColumnarToRow
 :  :  :  :                       +- CometExchange
 :  :  :  :                          +- CometHashAggregate
 :  :  :  :                             +- CometProject
 :  :  :  :                                +- CometFilter
 :  :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometColumnarExchange
 :  :  :              +- HashAggregate
 :  :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :  :                    +- CometNativeColumnarToRow
+:  :  :                    +- CometColumnarToRow
 :  :  :                       +- CometExchange
 :  :  :                          +- CometHashAggregate
 :  :  :                             +- CometProject
 :  :  :                                +- CometFilter
 :  :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometColumnarExchange
 :  :              +- HashAggregate
 :  :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:  :                    +- CometNativeColumnarToRow
+:  :                    +- CometColumnarToRow
 :  :                       +- CometExchange
 :  :                          +- CometHashAggregate
 :  :                             +- CometProject
 :  :                                +- CometFilter
 :  :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometColumnarExchange
 :              +- HashAggregate
 :                 +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-:                    +- CometNativeColumnarToRow
+:                    +- CometColumnarToRow
 :                       +- CometExchange
 :                          +- CometHashAggregate
 :                             +- CometProject
 :                                +- CometFilter
 :                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometColumnarExchange
             +- HashAggregate
                +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometExchange
                         +- CometHashAggregate
                            +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
index 9ca9f72ab4..8c47903ee8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
index 8b2f687037..70bff8abb7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
index 3318935f4d..780504fb09 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
index 60c49121a6..c7273327fd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
index 638a87ab14..971bf3c1f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
index e434c5ff76..2bf9a90726 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
index a92eb1903d..be3d6196b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
index cb4d06350b..715ff39db7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
index e8f621d32a..8a5a4af707 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
index 570f9bbcc9..2336733a6a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
index f9c807168e..8d2e593975 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
index 1272f205e4..cb125e2456 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
index 259056f9f6..0952fbf8d5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt
@@ -9,12 +9,12 @@ TakeOrderedAndProject
       :     :     :  +- Project
       :     :     :     +- Filter
       :     :     :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :     :           +- CometNativeColumnarToRow
+      :     :     :           +- CometColumnarToRow
       :     :     :              +- CometSort
       :     :     :                 +- CometExchange
       :     :     :                    +- CometFilter
       :     :     :                       :  +- Subquery
-      :     :     :                       :     +- CometNativeColumnarToRow
+      :     :     :                       :     +- CometColumnarToRow
       :     :     :                       :        +- CometHashAggregate
       :     :     :                       :           +- CometExchange
       :     :     :                       :              +- CometHashAggregate
@@ -31,12 +31,12 @@ TakeOrderedAndProject
       :     :        +- Project
       :     :           +- Filter
       :     :              +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :                 +- CometNativeColumnarToRow
+      :     :                 +- CometColumnarToRow
       :     :                    +- CometSort
       :     :                       +- CometExchange
       :     :                          +- CometFilter
       :     :                             :  +- Subquery
-      :     :                             :     +- CometNativeColumnarToRow
+      :     :                             :     +- CometColumnarToRow
       :     :                             :        +- CometHashAggregate
       :     :                             :           +- CometExchange
       :     :                             :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
index afdf8d6633..23662b8aaf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
                +- Filter
                   +-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
index c8325b49ca..347702b724 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
index 4b840c6a27..76c166e7de 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
index aa0c4c1c38..958b844c9e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
index ea746c5fee..8a2badfaff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometProject
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                      :  +- CometColumnarExchange
                      :     +- Project
                      :        +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                     :           +- CometNativeColumnarToRow
+                     :           +- CometColumnarToRow
                      :              +- CometSort
                      :                 +- CometExchange
                      :                    +- CometHashAggregate
@@ -34,7 +34,7 @@ TakeOrderedAndProject
                         +- CometColumnarExchange
                            +- Project
                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                 +- CometNativeColumnarToRow
+                                 +- CometColumnarToRow
                                     +- CometSort
                                        +- CometExchange
                                           +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
index f4318de30f..0b482769c1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
index e5122dc8d7..a292badf5d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
index 81412250ec..e503b00314 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
index 3594fb8035..3d50a2131e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
index e3297b4d66..20115b61c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
index faef2d5263..22c07c4964 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
index 783de88195..f359e64d7b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -48,7 +48,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
index dc09a1507f..a5fd7359a3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
index b0c27d6da7..755be00004 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
index 46b1956f4e..7e7f77c2c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
index da0bef861d..0ba8f6b0e7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
index 07093ae00b..28f2b6110d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
index bf624b5ce3..5a472217df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -10,7 +10,7 @@ TakeOrderedAndProject
                   :     :- Project
                   :     :  +- BroadcastHashJoin
                   :     :     :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-                  :     :     :  :- CometNativeColumnarToRow
+                  :     :     :  :- CometColumnarToRow
                   :     :     :  :  +- CometBroadcastHashJoin
                   :     :     :  :     :- CometFilter
                   :     :     :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -29,7 +29,7 @@ TakeOrderedAndProject
                   :     :     :  :                    +- CometFilter
                   :     :     :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     :  +- BroadcastExchange
-                  :     :     :     +- CometNativeColumnarToRow
+                  :     :     :     +- CometColumnarToRow
                   :     :     :        +- CometProject
                   :     :     :           +- CometBroadcastHashJoin
                   :     :     :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                   :     :     :                    +- CometFilter
                   :     :     :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :     +- BroadcastExchange
-                  :     :        +- CometNativeColumnarToRow
+                  :     :        +- CometColumnarToRow
                   :     :           +- CometProject
                   :     :              +- CometBroadcastHashJoin
                   :     :                 :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
index 783b9a64bb..3aa5ad7551 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
index 14e4cb087a..fa1bf8daa4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt
@@ -1,17 +1,17 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometColumnarExchange
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- Expand
                               +- Project
                                  +- BroadcastHashJoin
-                                    :- CometNativeColumnarToRow
+                                    :- CometColumnarToRow
                                     :  +- CometProject
                                     :     +- CometBroadcastHashJoin
                                     :        :- CometFilter
@@ -36,7 +36,7 @@ TakeOrderedAndProject
                                                 +- Project
                                                    +- Filter
                                                       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                         +- CometNativeColumnarToRow
+                                                         +- CometColumnarToRow
                                                             +- CometSort
                                                                +- CometHashAggregate
                                                                   +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
index 4de5ab6c3b..12dc136fee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
index 685ff36fed..b56a6590db 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
index 0cfc0a8417..a16105c248 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Union
-                  :- CometNativeColumnarToRow
+                  :- CometColumnarToRow
                   :  +- CometProject
                   :     +- CometBroadcastHashJoin
                   :        :- CometHashAggregate
@@ -51,7 +51,7 @@ TakeOrderedAndProject
                   :- Project
                   :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                   :     :- BroadcastExchange
-                  :     :  +- CometNativeColumnarToRow
+                  :     :  +- CometColumnarToRow
                   :     :     +- CometHashAggregate
                   :     :        +- CometExchange
                   :     :           +- CometHashAggregate
@@ -63,7 +63,7 @@ TakeOrderedAndProject
                   :     :                       +- CometProject
                   :     :                          +- CometFilter
                   :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  :     +- CometNativeColumnarToRow
+                  :     +- CometColumnarToRow
                   :        +- CometHashAggregate
                   :           +- CometExchange
                   :              +- CometHashAggregate
@@ -75,7 +75,7 @@ TakeOrderedAndProject
                   :                          +- CometProject
                   :                             +- CometFilter
                   :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometProject
                         +- CometBroadcastHashJoin
                            :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
index f3e80e01cb..871cf47610 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
index be5abf4913..4d7a7d1ce7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
index ed8d0deb6d..dd2aed67bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
index eb965c592c..731fcea912 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
index 17e0a010c0..2ab6dd2d0c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
index 3e72e3d553..9ab4ae91f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
index fcd8384c22..5fce933878 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
index f99808facc..5c782b95f8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
index c0056e2382..6444b727e4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
index 5fddd74768..0c51ae09a5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt
@@ -1,11 +1,11 @@
 HashAggregate
-+- CometNativeColumnarToRow
++- CometColumnarToRow
    +- CometColumnarExchange
       +- HashAggregate
          +- Project
             +- BroadcastHashJoin
                :-  BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported]
-               :  :- CometNativeColumnarToRow
+               :  :- CometColumnarToRow
                :  :  +- CometHashAggregate
                :  :     +- CometExchange
                :  :        +- CometHashAggregate
@@ -30,7 +30,7 @@ HashAggregate
                :  :                       +- CometFilter
                :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                :  +- BroadcastExchange
-               :     +- CometNativeColumnarToRow
+               :     +- CometColumnarToRow
                :        +- CometHashAggregate
                :           +- CometExchange
                :              +- CometHashAggregate
@@ -50,7 +50,7 @@ HashAggregate
                :                             +- CometFilter
                :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
                +- BroadcastExchange
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometHashAggregate
                         +- CometExchange
                            +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
index f1e4c8679a..1e1247665c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ BroadcastNestedLoopJoin
 :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :- BroadcastNestedLoopJoin
 :  :  :  :  :  :-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-:  :  :  :  :  :  :- CometNativeColumnarToRow
+:  :  :  :  :  :  :- CometColumnarToRow
 :  :  :  :  :  :  :  +- CometHashAggregate
 :  :  :  :  :  :  :     +- CometExchange
 :  :  :  :  :  :  :        +- CometHashAggregate
@@ -31,7 +31,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :  :                       +- CometFilter
 :  :  :  :  :  :  :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :  :           +- CometExchange
 :  :  :  :  :  :              +- CometHashAggregate
@@ -57,7 +57,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :  :                             +- CometFilter
 :  :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  :  +- BroadcastExchange
-:  :  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :  :     +- CometColumnarToRow
 :  :  :  :  :        +- CometHashAggregate
 :  :  :  :  :           +- CometExchange
 :  :  :  :  :              +- CometHashAggregate
@@ -83,7 +83,7 @@ BroadcastNestedLoopJoin
 :  :  :  :  :                             +- CometFilter
 :  :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  :  +- BroadcastExchange
-:  :  :  :     +- CometNativeColumnarToRow
+:  :  :  :     +- CometColumnarToRow
 :  :  :  :        +- CometHashAggregate
 :  :  :  :           +- CometExchange
 :  :  :  :              +- CometHashAggregate
@@ -109,7 +109,7 @@ BroadcastNestedLoopJoin
 :  :  :  :                             +- CometFilter
 :  :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  :  +- BroadcastExchange
-:  :  :     +- CometNativeColumnarToRow
+:  :  :     +- CometColumnarToRow
 :  :  :        +- CometHashAggregate
 :  :  :           +- CometExchange
 :  :  :              +- CometHashAggregate
@@ -135,7 +135,7 @@ BroadcastNestedLoopJoin
 :  :  :                             +- CometFilter
 :  :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  :  +- BroadcastExchange
-:  :     +- CometNativeColumnarToRow
+:  :     +- CometColumnarToRow
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
 :  :              +- CometHashAggregate
@@ -161,7 +161,7 @@ BroadcastNestedLoopJoin
 :  :                             +- CometFilter
 :  :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 :  +- BroadcastExchange
-:     +- CometNativeColumnarToRow
+:     +- CometColumnarToRow
 :        +- CometHashAggregate
 :           +- CometExchange
 :              +- CometHashAggregate
@@ -187,7 +187,7 @@ BroadcastNestedLoopJoin
 :                             +- CometFilter
 :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store
 +- BroadcastExchange
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometHashAggregate
          +- CometExchange
             +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
index 093505a6cc..6b4c3e3383 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt
@@ -2,7 +2,7 @@ TakeOrderedAndProject
 +- Project
    +- Filter
       +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometExchange
                   +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
index e3614131fe..a86d85c820 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
  Project [COMET: ]
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -11,7 +11,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -22,7 +22,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -33,7 +33,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
@@ -44,7 +44,7 @@
 :  :- ReusedSubquery
 :  :- ReusedSubquery
 :  :- Subquery
-:  :  +- CometNativeColumnarToRow
+:  :  +- CometColumnarToRow
 :  :     +- CometProject
 :  :        +- CometHashAggregate
 :  :           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
index 3b1bd00423..aa6c577ed7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 Project
 +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-   :- CometNativeColumnarToRow
+   :- CometColumnarToRow
    :  +- CometHashAggregate
    :     +- CometExchange
    :        +- CometHashAggregate
@@ -26,7 +26,7 @@ Project
    :                       +- CometFilter
    :                          +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
    +- BroadcastExchange
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometHashAggregate
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
index 961299b3d9..ff13331764 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
index 0fb6e6c158..ca50f78e18 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
index 79acf0e4eb..335f2765d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
index ff3a1a06ad..eac4939621 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
index bcb3e1d444..6ff8eba58f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometColumnarExchange
       +- HashAggregate
          +-  HashAggregate [COMET: Unsupported aggregation mode PartialMerge]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometHashAggregate
                   +- CometProject
                      +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
index 0f07a6473f..0f623c9021 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
index 4aa20a3755..0969a0e796 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometHashAggregate
    +- CometExchange
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
index a18137ef10..152665febf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt
@@ -1,10 +1,10 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometProject
    +- CometSort
       +- CometColumnarExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
index 47e040af97..86cf2fc2f3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
index 78557e7ffb..c1b791f357 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
index a8adb0443d..3bacf3c8bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometNativeColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometNativeColumnarToRow
+               :           :  :     +- CometColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometNativeColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometNativeColumnarToRow
+                                    :  :     +- CometColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
index 94f7834f85..148f6d4a6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
index f642ebc712..5da4848ebe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
index d83496133a..4ccef655b9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
index 5d8bc6a064..91b709aa47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometNativeColumnarToRow
+         :        +- CometColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometNativeColumnarToRow
+         :                    +- CometColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometNativeColumnarToRow
+         :                                   :- CometColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometNativeColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometNativeColumnarToRow
+                                       :- CometColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
index e7288e7e47..a3e6b17491 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
index e9571a1f12..968ea0611f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
index 6560795de1..05c8752bd2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometNativeColumnarToRow
+               :                    :        +- CometColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometNativeColumnarToRow
+               :                    :                       :     +- CometColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometNativeColumnarToRow
+               :                    :                                +- CometColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometNativeColumnarToRow
+               :                             +- CometColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometNativeColumnarToRow
+               :                                            :     +- CometColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometNativeColumnarToRow
+               :                                                     +- CometColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometNativeColumnarToRow
+                                          :        +- CometColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometNativeColumnarToRow
+                                          :                       :     +- CometColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometNativeColumnarToRow
+                                          :                                +- CometColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometNativeColumnarToRow
+                                                   +- CometColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometNativeColumnarToRow
+                                                                  :     +- CometColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometNativeColumnarToRow
+                                                                           +- CometColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
index e2832782b8..24167a2372 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
index e3297b4d66..20115b61c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
index 2afcab203a..407fc9836c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometNativeColumnarToRow
+                     +- CometColumnarToRow
                         +- CometSort
                            +- CometUnion
                               :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
index b21df154e6..0ab48d6a12 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometNativeColumnarToRow
+                           :                 :- CometColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                            :                                +- Filter
                            :                                   +- Window
                            :                                      +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                         +- CometNativeColumnarToRow
+                           :                                         +- CometColumnarToRow
                            :                                            +- CometSort
                            :                                               +- CometHashAggregate
                            :                                                  +- CometExchange
@@ -61,16 +61,16 @@ TakeOrderedAndProject
                            :                                                                    +- CometFilter
                            :                                                                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometNativeColumnarToRow
+                           :              +- CometColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometNativeColumnarToRow
+                           :                             :- CometColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -96,7 +96,7 @@ TakeOrderedAndProject
                            :                                            +- Filter
                            :                                               +- Window
                            :                                                  +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                                     +- CometNativeColumnarToRow
+                           :                                                     +- CometColumnarToRow
                            :                                                        +- CometSort
                            :                                                           +- CometHashAggregate
                            :                                                              +- CometExchange
@@ -117,16 +117,16 @@ TakeOrderedAndProject
                            :                                                                                +- CometFilter
                            :                                                                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometNativeColumnarToRow
+                                                         :- CometColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -152,7 +152,7 @@ TakeOrderedAndProject
                                                                         +- Filter
                                                                            +- Window
                                                                               +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                                                 +- CometNativeColumnarToRow
+                                                                                 +- CometColumnarToRow
                                                                                     +- CometSort
                                                                                        +- CometHashAggregate
                                                                                           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
index 3578f807aa..40e2c31863 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometNativeColumnarToRow
+               :              :- CometColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometNativeColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometNativeColumnarToRow
+               :              :     :  +- CometColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometNativeColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometNativeColumnarToRow
+               :              :     +- CometColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometNativeColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometNativeColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometNativeColumnarToRow
+               :                          :- CometColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometNativeColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometNativeColumnarToRow
+               :                          :     :  +- CometColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometNativeColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometNativeColumnarToRow
+               :                          :     +- CometColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometNativeColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometNativeColumnarToRow
+               :                          +- CometColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometNativeColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometNativeColumnarToRow
+                                          :- CometColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometNativeColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometNativeColumnarToRow
+                                          :     :  +- CometColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometNativeColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometNativeColumnarToRow
+                                          :     +- CometColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometNativeColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
index fbd7cba46d..943d6e8d1a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
index 58d00a427d..13ff1e34cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
index 1a985ead1b..91e569a85c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
index 7a4afd3d2b..49eafd1015 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
index a8adb0443d..3bacf3c8bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometNativeColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometNativeColumnarToRow
+               :           :  :     +- CometColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometNativeColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometNativeColumnarToRow
+                                    :  :     +- CometColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
index 94f7834f85..148f6d4a6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
index f642ebc712..5da4848ebe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
index d83496133a..4ccef655b9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
index 5d8bc6a064..91b709aa47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometNativeColumnarToRow
+         :        +- CometColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometNativeColumnarToRow
+         :                    +- CometColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometNativeColumnarToRow
+         :                                   :- CometColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometNativeColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometNativeColumnarToRow
+                                       :- CometColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
index e7288e7e47..a3e6b17491 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
index e9571a1f12..968ea0611f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
index 6560795de1..05c8752bd2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometNativeColumnarToRow
+               :                    :        +- CometColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometNativeColumnarToRow
+               :                    :                       :     +- CometColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometNativeColumnarToRow
+               :                    :                                +- CometColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometNativeColumnarToRow
+               :                             +- CometColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometNativeColumnarToRow
+               :                                            :     +- CometColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometNativeColumnarToRow
+               :                                                     +- CometColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometNativeColumnarToRow
+                                          :        +- CometColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometNativeColumnarToRow
+                                          :                       :     +- CometColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometNativeColumnarToRow
+                                          :                                +- CometColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometNativeColumnarToRow
+                                                   +- CometColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometNativeColumnarToRow
+                                                                  :     +- CometColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometNativeColumnarToRow
+                                                                           +- CometColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
index e2832782b8..24167a2372 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
index ee55822981..4bb14d3c19 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
index 2afcab203a..407fc9836c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt
@@ -2,11 +2,11 @@ TakeOrderedAndProject
 +- Filter
    +- Window
       +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-         +- CometNativeColumnarToRow
+         +- CometColumnarToRow
             +- CometSort
                +- CometColumnarExchange
                   +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                     +- CometNativeColumnarToRow
+                     +- CometColumnarToRow
                         +- CometSort
                            +- CometUnion
                               :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
index b21df154e6..0ab48d6a12 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometNativeColumnarToRow
+                           :                 :- CometColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                            :                                +- Filter
                            :                                   +- Window
                            :                                      +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                         +- CometNativeColumnarToRow
+                           :                                         +- CometColumnarToRow
                            :                                            +- CometSort
                            :                                               +- CometHashAggregate
                            :                                                  +- CometExchange
@@ -61,16 +61,16 @@ TakeOrderedAndProject
                            :                                                                    +- CometFilter
                            :                                                                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometNativeColumnarToRow
+                           :              +- CometColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometNativeColumnarToRow
+                           :                             :- CometColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -96,7 +96,7 @@ TakeOrderedAndProject
                            :                                            +- Filter
                            :                                               +- Window
                            :                                                  +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                           :                                                     +- CometNativeColumnarToRow
+                           :                                                     +- CometColumnarToRow
                            :                                                        +- CometSort
                            :                                                           +- CometHashAggregate
                            :                                                              +- CometExchange
@@ -117,16 +117,16 @@ TakeOrderedAndProject
                            :                                                                                +- CometFilter
                            :                                                                                   +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometNativeColumnarToRow
+                                                         :- CometColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -152,7 +152,7 @@ TakeOrderedAndProject
                                                                         +- Filter
                                                                            +- Window
                                                                               +-  WindowGroupLimit [COMET: WindowGroupLimit is not supported]
-                                                                                 +- CometNativeColumnarToRow
+                                                                                 +- CometColumnarToRow
                                                                                     +- CometSort
                                                                                        +- CometHashAggregate
                                                                                           +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
index 3578f807aa..40e2c31863 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometNativeColumnarToRow
+               :              :- CometColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometNativeColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometNativeColumnarToRow
+               :              :     :  +- CometColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometNativeColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometNativeColumnarToRow
+               :              :     +- CometColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometNativeColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometNativeColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometNativeColumnarToRow
+               :                          :- CometColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometNativeColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometNativeColumnarToRow
+               :                          :     :  +- CometColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometNativeColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometNativeColumnarToRow
+               :                          :     +- CometColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometNativeColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometNativeColumnarToRow
+               :                          +- CometColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometNativeColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometNativeColumnarToRow
+                                          :- CometColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometNativeColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometNativeColumnarToRow
+                                          :     :  +- CometColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometNativeColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometNativeColumnarToRow
+                                          :     +- CometColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometNativeColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
index fbd7cba46d..943d6e8d1a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
index 58d00a427d..13ff1e34cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
index 1a985ead1b..91e569a85c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
index eb85a80eb1..1812b9e872 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
index 78557e7ffb..c1b791f357 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometBroadcastHashJoin
       :- CometFilter
       :  :  +- Subquery
-      :  :     +- CometNativeColumnarToRow
+      :  :     +- CometColumnarToRow
       :  :        +- CometHashAggregate
       :  :           +- CometExchange
       :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
index a8adb0443d..3bacf3c8bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
@@ -10,7 +10,7 @@ CometNativeColumnarToRow
                :        +- CometUnion
                :           :- CometFilter
                :           :  :  +- Subquery
-               :           :  :     +- CometNativeColumnarToRow
+               :           :  :     +- CometColumnarToRow
                :           :  :        +- CometHashAggregate
                :           :  :           +- CometExchange
                :           :  :              +- CometHashAggregate
@@ -479,7 +479,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -948,7 +948,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1417,7 +1417,7 @@ CometNativeColumnarToRow
                :                 +- CometUnion
                :                    :- CometFilter
                :                    :  :  +- Subquery
-               :                    :  :     +- CometNativeColumnarToRow
+               :                    :  :     +- CometColumnarToRow
                :                    :  :        +- CometHashAggregate
                :                    :  :           +- CometExchange
                :                    :  :              +- CometHashAggregate
@@ -1886,7 +1886,7 @@ CometNativeColumnarToRow
                                  +- CometUnion
                                     :- CometFilter
                                     :  :  +- Subquery
-                                    :  :     +- CometNativeColumnarToRow
+                                    :  :     +- CometColumnarToRow
                                     :  :        +- CometHashAggregate
                                     :  :           +- CometExchange
                                     :  :              +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
index 94f7834f85..148f6d4a6d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
index 643e47b0c6..0cfd81f36f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
index f642ebc712..5da4848ebe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Expand
                +- Project
                   +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
-                     :- CometNativeColumnarToRow
+                     :- CometColumnarToRow
                      :  +- CometProject
                      :     +- CometBroadcastHashJoin
                      :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
index d83496133a..4ccef655b9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
index 5d8bc6a064..91b709aa47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt
@@ -1,19 +1,19 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Filter
          :  +- Subquery
          :     +- HashAggregate
-         :        +- CometNativeColumnarToRow
+         :        +- CometColumnarToRow
          :           +- CometColumnarExchange
          :              +- HashAggregate
          :                 +- HashAggregate
-         :                    +- CometNativeColumnarToRow
+         :                    +- CometColumnarToRow
          :                       +- CometColumnarExchange
          :                          +- HashAggregate
          :                             +- Project
          :                                +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-         :                                   :- CometNativeColumnarToRow
+         :                                   :- CometColumnarToRow
          :                                   :  +- CometProject
          :                                   :     +- CometBroadcastHashJoin
          :                                   :        :- CometProject
@@ -50,16 +50,16 @@ CometNativeColumnarToRow
          :                                            +- CometFilter
          :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address
          +- HashAggregate
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometColumnarExchange
                   +- HashAggregate
                      +- HashAggregate
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometColumnarExchange
                               +- HashAggregate
                                  +- Project
                                     +-  BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.]
-                                       :- CometNativeColumnarToRow
+                                       :- CometColumnarToRow
                                        :  +- CometProject
                                        :     +- CometBroadcastHashJoin
                                        :        :- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
index e7288e7e47..a3e6b17491 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometUnion
       :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
index 18ed3e6335..65e639c1d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
index c5125b1861..855ca74e0b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +- HashAggregate
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometColumnarExchange
          +- HashAggregate
             +- Project
@@ -11,7 +11,7 @@ TakeOrderedAndProject
                   :     :  +- Filter
                   :     :     +- BroadcastHashJoin
                   :     :        :-  BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
-                  :     :        :  :- CometNativeColumnarToRow
+                  :     :        :  :- CometColumnarToRow
                   :     :        :  :  +- CometBroadcastHashJoin
                   :     :        :  :     :- CometFilter
                   :     :        :  :     :  +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer
@@ -30,7 +30,7 @@ TakeOrderedAndProject
                   :     :        :  :                    +- CometFilter
                   :     :        :  :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        :  +- BroadcastExchange
-                  :     :        :     +- CometNativeColumnarToRow
+                  :     :        :     +- CometColumnarToRow
                   :     :        :        +- CometProject
                   :     :        :           +- CometBroadcastHashJoin
                   :     :        :              :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales
@@ -40,7 +40,7 @@ TakeOrderedAndProject
                   :     :        :                    +- CometFilter
                   :     :        :                       +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                   :     :        +- BroadcastExchange
-                  :     :           +- CometNativeColumnarToRow
+                  :     :           +- CometColumnarToRow
                   :     :              +- CometProject
                   :     :                 +- CometBroadcastHashJoin
                   :     :                    :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
index bdbfb0d6dc..fc330dd8b6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
index e9571a1f12..968ea0611f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
index 9691120d2f..f38ceaf317 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
index 0b4672eabb..d1b415dfe4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
@@ -9,7 +9,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -41,7 +41,7 @@ CometNativeColumnarToRow
                :     +- Window
                :        +- Sort
                :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometSort
                :                    +- CometExchange
                :                       +- CometHashAggregate
@@ -68,7 +68,7 @@ CometNativeColumnarToRow
                      +- Window
                         +- Sort
                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometSort
                                     +- CometExchange
                                        +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
index 6560795de1..05c8752bd2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt
@@ -5,7 +5,7 @@ TakeOrderedAndProject
          +- Project
             +- BroadcastHashJoin
                :-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometSort
                :        +- CometExchange
                :           +- CometProject
@@ -14,14 +14,14 @@ TakeOrderedAndProject
                :                    :- CometSort
                :                    :  +- CometColumnarExchange
                :                    :     +- HashAggregate
-               :                    :        +- CometNativeColumnarToRow
+               :                    :        +- CometColumnarToRow
                :                    :           +- CometColumnarExchange
                :                    :              +- HashAggregate
                :                    :                 +- Project
                :                    :                    +- BroadcastHashJoin
                :                    :                       :- Project
                :                    :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                       :     +- CometNativeColumnarToRow
+               :                    :                       :     +- CometColumnarToRow
                :                    :                       :        +- CometSort
                :                    :                       :           +- CometExchange
                :                    :                       :              +- CometHashAggregate
@@ -44,7 +44,7 @@ TakeOrderedAndProject
                :                    :                       +- BroadcastExchange
                :                    :                          +- Project
                :                    :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                    :                                +- CometNativeColumnarToRow
+               :                    :                                +- CometColumnarToRow
                :                    :                                   +- CometSort
                :                    :                                      +- CometExchange
                :                    :                                         +- CometHashAggregate
@@ -67,14 +67,14 @@ TakeOrderedAndProject
                :                    +- CometSort
                :                       +- CometColumnarExchange
                :                          +- HashAggregate
-               :                             +- CometNativeColumnarToRow
+               :                             +- CometColumnarToRow
                :                                +- CometColumnarExchange
                :                                   +- HashAggregate
                :                                      +- Project
                :                                         +- BroadcastHashJoin
                :                                            :- Project
                :                                            :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                            :     +- CometNativeColumnarToRow
+               :                                            :     +- CometColumnarToRow
                :                                            :        +- CometSort
                :                                            :           +- CometExchange
                :                                            :              +- CometHashAggregate
@@ -92,7 +92,7 @@ TakeOrderedAndProject
                :                                            +- BroadcastExchange
                :                                               +- Project
                :                                                  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               :                                                     +- CometNativeColumnarToRow
+               :                                                     +- CometColumnarToRow
                :                                                        +- CometSort
                :                                                           +- CometExchange
                :                                                              +- CometHashAggregate
@@ -110,7 +110,7 @@ TakeOrderedAndProject
                +- BroadcastExchange
                   +- Project
                      +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                        +- CometNativeColumnarToRow
+                        +- CometColumnarToRow
                            +- CometSort
                               +- CometExchange
                                  +- CometProject
@@ -119,14 +119,14 @@ TakeOrderedAndProject
                                           :- CometSort
                                           :  +- CometColumnarExchange
                                           :     +- HashAggregate
-                                          :        +- CometNativeColumnarToRow
+                                          :        +- CometColumnarToRow
                                           :           +- CometColumnarExchange
                                           :              +- HashAggregate
                                           :                 +- Project
                                           :                    +- BroadcastHashJoin
                                           :                       :- Project
                                           :                       :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                       :     +- CometNativeColumnarToRow
+                                          :                       :     +- CometColumnarToRow
                                           :                       :        +- CometSort
                                           :                       :           +- CometExchange
                                           :                       :              +- CometHashAggregate
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                           :                       +- BroadcastExchange
                                           :                          +- Project
                                           :                             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                          :                                +- CometNativeColumnarToRow
+                                          :                                +- CometColumnarToRow
                                           :                                   +- CometSort
                                           :                                      +- CometExchange
                                           :                                         +- CometHashAggregate
@@ -172,14 +172,14 @@ TakeOrderedAndProject
                                           +- CometSort
                                              +- CometColumnarExchange
                                                 +- HashAggregate
-                                                   +- CometNativeColumnarToRow
+                                                   +- CometColumnarToRow
                                                       +- CometColumnarExchange
                                                          +- HashAggregate
                                                             +- Project
                                                                +- BroadcastHashJoin
                                                                   :- Project
                                                                   :  +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                  :     +- CometNativeColumnarToRow
+                                                                  :     +- CometColumnarToRow
                                                                   :        +- CometSort
                                                                   :           +- CometExchange
                                                                   :              +- CometHashAggregate
@@ -197,7 +197,7 @@ TakeOrderedAndProject
                                                                   +- BroadcastExchange
                                                                      +- Project
                                                                         +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                           +- CometNativeColumnarToRow
+                                                                           +- CometColumnarToRow
                                                                               +- CometSort
                                                                                  +- CometExchange
                                                                                     +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
index 5e8acff889..1c37fd4e8c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject
       :     :     +- Window
       :     :        +- Filter
       :     :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :     :              +- CometNativeColumnarToRow
+      :     :              +- CometColumnarToRow
       :     :                 +- CometSort
       :     :                    +- CometExchange
       :     :                       +- CometHashAggregate
@@ -40,7 +40,7 @@ TakeOrderedAndProject
       :     +- BroadcastExchange
       :        +- Project
       :           +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      :              +- CometNativeColumnarToRow
+      :              +- CometColumnarToRow
       :                 +- CometSort
       :                    +- CometExchange
       :                       +- CometHashAggregate
@@ -72,7 +72,7 @@ TakeOrderedAndProject
       +- BroadcastExchange
          +- Project
             +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-               +- CometNativeColumnarToRow
+               +- CometColumnarToRow
                   +- CometSort
                      +- CometExchange
                         +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
index e2832782b8..24167a2372 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
index e3297b4d66..20115b61c4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometFilter
       +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
index a868670a16..75884c3f9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometExchange
       +- CometProject
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
index b610537f40..c5c2f024b9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Filter
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometUnion
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
index a75ecb0e47..94b23c3b4f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
@@ -9,12 +9,12 @@ TakeOrderedAndProject
                      +- HashAggregate
                         +- Union
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- Project
                            :              +- BroadcastHashJoin
-                           :                 :- CometNativeColumnarToRow
+                           :                 :- CometColumnarToRow
                            :                 :  +- CometProject
                            :                 :     +- CometBroadcastHashJoin
                            :                 :        :- CometFilter
@@ -39,7 +39,7 @@ TakeOrderedAndProject
                            :                             +- Project
                            :                                +- Filter
                            :                                   +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                           :                                      +- CometNativeColumnarToRow
+                           :                                      +- CometColumnarToRow
                            :                                         +- CometSort
                            :                                            +- CometHashAggregate
                            :                                               +- CometExchange
@@ -60,16 +60,16 @@ TakeOrderedAndProject
                            :                                                                 +- CometFilter
                            :                                                                    +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            :- HashAggregate
-                           :  +- CometNativeColumnarToRow
+                           :  +- CometColumnarToRow
                            :     +- CometColumnarExchange
                            :        +- HashAggregate
                            :           +- HashAggregate
-                           :              +- CometNativeColumnarToRow
+                           :              +- CometColumnarToRow
                            :                 +- CometColumnarExchange
                            :                    +- HashAggregate
                            :                       +- Project
                            :                          +- BroadcastHashJoin
-                           :                             :- CometNativeColumnarToRow
+                           :                             :- CometColumnarToRow
                            :                             :  +- CometProject
                            :                             :     +- CometBroadcastHashJoin
                            :                             :        :- CometFilter
@@ -94,7 +94,7 @@ TakeOrderedAndProject
                            :                                         +- Project
                            :                                            +- Filter
                            :                                               +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                           :                                                  +- CometNativeColumnarToRow
+                           :                                                  +- CometColumnarToRow
                            :                                                     +- CometSort
                            :                                                        +- CometHashAggregate
                            :                                                           +- CometExchange
@@ -115,16 +115,16 @@ TakeOrderedAndProject
                            :                                                                             +- CometFilter
                            :                                                                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- HashAggregate
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometColumnarExchange
                                                 +- HashAggregate
                                                    +- Project
                                                       +- BroadcastHashJoin
-                                                         :- CometNativeColumnarToRow
+                                                         :- CometColumnarToRow
                                                          :  +- CometProject
                                                          :     +- CometBroadcastHashJoin
                                                          :        :- CometFilter
@@ -149,7 +149,7 @@ TakeOrderedAndProject
                                                                      +- Project
                                                                         +- Filter
                                                                            +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-                                                                              +- CometNativeColumnarToRow
+                                                                              +- CometColumnarToRow
                                                                                  +- CometSort
                                                                                     +- CometHashAggregate
                                                                                        +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
index 22fde3b02c..819698e99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
index 922864baa1..9fb1ccfbe0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometBroadcastHashJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
index ec53af4809..a3bbeab3af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometProject
       +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
index 3578f807aa..40e2c31863 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt
@@ -1,15 +1,15 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometColumnarExchange
          +- HashAggregate
             +- Union
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- Union
-               :              :- CometNativeColumnarToRow
+               :              :- CometColumnarToRow
                :              :  +- CometProject
                :              :     +- CometBroadcastHashJoin
                :              :        :- CometHashAggregate
@@ -55,7 +55,7 @@ CometNativeColumnarToRow
                :              :- Project
                :              :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :              :     :- BroadcastExchange
-               :              :     :  +- CometNativeColumnarToRow
+               :              :     :  +- CometColumnarToRow
                :              :     :     +- CometHashAggregate
                :              :     :        +- CometExchange
                :              :     :           +- CometHashAggregate
@@ -67,7 +67,7 @@ CometNativeColumnarToRow
                :              :     :                       +- CometProject
                :              :     :                          +- CometFilter
                :              :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              :     +- CometNativeColumnarToRow
+               :              :     +- CometColumnarToRow
                :              :        +- CometHashAggregate
                :              :           +- CometExchange
                :              :              +- CometHashAggregate
@@ -79,7 +79,7 @@ CometNativeColumnarToRow
                :              :                          +- CometProject
                :              :                             +- CometFilter
                :              :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometProject
                :                    +- CometBroadcastHashJoin
                :                       :- CometHashAggregate
@@ -118,15 +118,15 @@ CometNativeColumnarToRow
                :                                            +- CometFilter
                :                                               +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                :- HashAggregate
-               :  +- CometNativeColumnarToRow
+               :  +- CometColumnarToRow
                :     +- CometColumnarExchange
                :        +- HashAggregate
                :           +- HashAggregate
-               :              +- CometNativeColumnarToRow
+               :              +- CometColumnarToRow
                :                 +- CometColumnarExchange
                :                    +- HashAggregate
                :                       +- Union
-               :                          :- CometNativeColumnarToRow
+               :                          :- CometColumnarToRow
                :                          :  +- CometProject
                :                          :     +- CometBroadcastHashJoin
                :                          :        :- CometHashAggregate
@@ -172,7 +172,7 @@ CometNativeColumnarToRow
                :                          :- Project
                :                          :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                :                          :     :- BroadcastExchange
-               :                          :     :  +- CometNativeColumnarToRow
+               :                          :     :  +- CometColumnarToRow
                :                          :     :     +- CometHashAggregate
                :                          :     :        +- CometExchange
                :                          :     :           +- CometHashAggregate
@@ -184,7 +184,7 @@ CometNativeColumnarToRow
                :                          :     :                       +- CometProject
                :                          :     :                          +- CometFilter
                :                          :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          :     +- CometNativeColumnarToRow
+               :                          :     +- CometColumnarToRow
                :                          :        +- CometHashAggregate
                :                          :           +- CometExchange
                :                          :              +- CometHashAggregate
@@ -196,7 +196,7 @@ CometNativeColumnarToRow
                :                          :                          +- CometProject
                :                          :                             +- CometFilter
                :                          :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-               :                          +- CometNativeColumnarToRow
+               :                          +- CometColumnarToRow
                :                             +- CometProject
                :                                +- CometBroadcastHashJoin
                :                                   :- CometHashAggregate
@@ -235,15 +235,15 @@ CometNativeColumnarToRow
                :                                                        +- CometFilter
                :                                                           +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page
                +- HashAggregate
-                  +- CometNativeColumnarToRow
+                  +- CometColumnarToRow
                      +- CometColumnarExchange
                         +- HashAggregate
                            +- HashAggregate
-                              +- CometNativeColumnarToRow
+                              +- CometColumnarToRow
                                  +- CometColumnarExchange
                                     +- HashAggregate
                                        +- Union
-                                          :- CometNativeColumnarToRow
+                                          :- CometColumnarToRow
                                           :  +- CometProject
                                           :     +- CometBroadcastHashJoin
                                           :        :- CometHashAggregate
@@ -289,7 +289,7 @@ CometNativeColumnarToRow
                                           :- Project
                                           :  +-  BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported]
                                           :     :- BroadcastExchange
-                                          :     :  +- CometNativeColumnarToRow
+                                          :     :  +- CometColumnarToRow
                                           :     :     +- CometHashAggregate
                                           :     :        +- CometExchange
                                           :     :           +- CometHashAggregate
@@ -301,7 +301,7 @@ CometNativeColumnarToRow
                                           :     :                       +- CometProject
                                           :     :                          +- CometFilter
                                           :     :                             +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          :     +- CometNativeColumnarToRow
+                                          :     +- CometColumnarToRow
                                           :        +- CometHashAggregate
                                           :           +- CometExchange
                                           :              +- CometHashAggregate
@@ -313,7 +313,7 @@ CometNativeColumnarToRow
                                           :                          +- CometProject
                                           :                             +- CometFilter
                                           :                                +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim
-                                          +- CometNativeColumnarToRow
+                                          +- CometColumnarToRow
                                              +- CometProject
                                                 +- CometBroadcastHashJoin
                                                    :- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
index 1b1e6d0cde..3b721ab107 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject
 +-  Project [COMET: Comet does not support Spark's BigDecimal rounding]
-   +- CometNativeColumnarToRow
+   +- CometColumnarToRow
       +- CometSortMergeJoin
          :- CometProject
          :  +- CometSortMergeJoin
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
index fbd7cba46d..943d6e8d1a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt
@@ -1,4 +1,4 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometTakeOrderedAndProject
    +- CometHashAggregate
       +- CometExchange
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
index 58d00a427d..13ff1e34cb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject
 +- Project
    +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-      +- CometNativeColumnarToRow
+      +- CometColumnarToRow
          +- CometSort
             +- CometExchange
                +- CometHashAggregate
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
index 1a985ead1b..91e569a85c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt
@@ -1,9 +1,9 @@
-CometNativeColumnarToRow
+CometColumnarToRow
 +- CometSort
    +- CometColumnarExchange
       +- Project
          +-  Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).]
-            +- CometNativeColumnarToRow
+            +- CometColumnarToRow
                +- CometSort
                   +- CometExchange
                      +- CometHashAggregate

From 81627a93b94c138a62c84ec122cb3fdbbdfd7e32 Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 18:32:57 -0600
Subject: [PATCH 6/7] Revert "remove arrow_ffi_safe flag"

This reverts commit 64184e5f7f8bdcfa2ee7d534682d051e243fec65.
---
 native/core/src/execution/operators/copy.rs   | 29 ++++++++++++++++---
 native/core/src/execution/operators/mod.rs    |  2 +-
 native/core/src/execution/operators/scan.rs   | 20 ++++++++++---
 native/core/src/execution/planner.rs          | 15 ++++++++--
 native/proto/src/proto/operator.proto         |  4 +--
 .../operator/CometDataWritingCommand.scala    |  1 +
 .../comet/serde/operator/CometSink.scala      | 14 +++++++++
 .../comet/CometBroadcastExchangeExec.scala    |  7 +++++
 .../sql/comet/CometLocalTableScanExec.scala   |  1 +
 9 files changed, 80 insertions(+), 13 deletions(-)

diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs
index 4c2ea67578..193c385c3c 100644
--- a/native/core/src/execution/operators/copy.rs
+++ b/native/core/src/execution/operators/copy.rs
@@ -22,8 +22,16 @@ use arrow::array::{downcast_dictionary_array, make_array, Array, ArrayRef, Mutab
 use arrow::datatypes::DataType;
 use arrow::error::ArrowError;
 
+#[derive(Debug, PartialEq, Clone)]
+pub enum CopyMode {
+    /// Perform a deep copy and also unpack dictionaries
+    UnpackOrDeepCopy,
+    /// Perform a clone and also unpack dictionaries
+    UnpackOrClone,
+}
+
 /// Copy an Arrow Array
-fn copy_array(array: &dyn Array) -> ArrayRef {
+pub(crate) fn copy_array(array: &dyn Array) -> ArrayRef {
     let capacity = array.len();
     let data = array.to_data();
 
@@ -54,8 +62,15 @@ fn copy_array(array: &dyn Array) -> ArrayRef {
     }
 }
 
-/// Unpack dictionary arrays to primitive type, or clone non-dictionary arrays.
-pub(crate) fn copy_or_unpack_array(array: &Arc) -> Result {
+/// Copy an Arrow Array or cast to primitive type if it is a dictionary array.
+/// This is used for `CopyExec` to copy/cast the input array. If the input array
+/// is a dictionary array, we will cast the dictionary array to primitive type
+/// (i.e., unpack the dictionary array) and copy the primitive array. If the input
+/// array is a primitive array, we simply copy the array.
+pub(crate) fn copy_or_unpack_array(
+    array: &Arc,
+    mode: &CopyMode,
+) -> Result {
     match array.data_type() {
         DataType::Dictionary(_, value_type) => {
             let options = CastOptions::default();
@@ -67,6 +82,12 @@ pub(crate) fn copy_or_unpack_array(array: &Arc) -> Result Ok(Arc::clone(array)),
+        _ => {
+            if mode == &CopyMode::UnpackOrDeepCopy {
+                Ok(copy_array(array))
+            } else {
+                Ok(Arc::clone(array))
+            }
+        }
     }
 }
diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs
index 2780a9edfc..07ee995367 100644
--- a/native/core/src/execution/operators/mod.rs
+++ b/native/core/src/execution/operators/mod.rs
@@ -21,7 +21,7 @@ use std::fmt::Debug;
 
 use jni::objects::GlobalRef;
 
-pub(crate) use copy::*;
+pub use copy::*;
 pub use iceberg_scan::*;
 pub use scan::*;
 
diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs
index ef55e20485..2543705fb0 100644
--- a/native/core/src/execution/operators/scan.rs
+++ b/native/core/src/execution/operators/scan.rs
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-use crate::execution::operators::copy_or_unpack_array;
+use crate::execution::operators::{copy_array, copy_or_unpack_array, CopyMode};
 use crate::{
     errors::CometError,
     execution::{
@@ -77,6 +77,8 @@ pub struct ScanExec {
     metrics: ExecutionPlanMetricsSet,
     /// Baseline metrics
     baseline_metrics: BaselineMetrics,
+    /// Whether native code can assume ownership of batches that it receives
+    arrow_ffi_safe: bool,
 }
 
 impl ScanExec {
@@ -85,6 +87,7 @@ impl ScanExec {
         input_source: Option>,
         input_source_description: &str,
         data_types: Vec,
+        arrow_ffi_safe: bool,
     ) -> Result {
         let metrics_set = ExecutionPlanMetricsSet::default();
         let baseline_metrics = BaselineMetrics::new(&metrics_set, 0);
@@ -111,6 +114,7 @@ impl ScanExec {
             metrics: metrics_set,
             baseline_metrics,
             schema,
+            arrow_ffi_safe,
         })
     }
 
@@ -143,6 +147,7 @@ impl ScanExec {
                 self.exec_context_id,
                 self.input_source.as_ref().unwrap().as_obj(),
                 self.data_types.len(),
+                self.arrow_ffi_safe,
             )?;
             *current_batch = Some(next_batch);
         }
@@ -157,6 +162,7 @@ impl ScanExec {
         exec_context_id: i64,
         iter: &JObject,
         num_cols: usize,
+        arrow_ffi_safe: bool,
     ) -> Result {
         if exec_context_id == TEST_EXEC_CONTEXT_ID {
             // This is a unit test. We don't need to call JNI.
@@ -219,9 +225,15 @@ impl ScanExec {
                 array
             };
 
-            // ownership of this array has been transferred to native
-            // but we still need to unpack dictionary arrays
-            let array = copy_or_unpack_array(&array)?;
+            let array = if arrow_ffi_safe {
+                // ownership of this array has been transferred to native
+                // but we still need to unpack dictionary arrays
+                copy_or_unpack_array(&array, &CopyMode::UnpackOrClone)?
+            } else {
+                // it is necessary to copy the array because the contents may be
+                // overwritten on the JVM side in the future
+                copy_array(&array)
+            };
 
             inputs.push(array);
 
diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs
index 68e426707c..bd37755922 100644
--- a/native/core/src/execution/planner.rs
+++ b/native/core/src/execution/planner.rs
@@ -1266,8 +1266,13 @@ impl PhysicalPlanner {
                     };
 
                 // The `ScanExec` operator will take actual arrays from Spark during execution
-                let scan =
-                    ScanExec::new(self.exec_context_id, input_source, &scan.source, data_types)?;
+                let scan = ScanExec::new(
+                    self.exec_context_id,
+                    input_source,
+                    &scan.source,
+                    data_types,
+                    scan.arrow_ffi_safe,
+                )?;
 
                 Ok((
                     vec![scan.clone()],
@@ -3651,6 +3656,7 @@ mod tests {
                     type_info: None,
                 }],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         };
 
@@ -3715,6 +3721,7 @@ mod tests {
                     type_info: None,
                 }],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         };
 
@@ -3919,6 +3926,7 @@ mod tests {
             op_struct: Some(OpStruct::Scan(spark_operator::Scan {
                 fields: vec![create_proto_datatype()],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         }
     }
@@ -3961,6 +3969,7 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         };
 
@@ -4082,6 +4091,7 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         };
 
@@ -4564,6 +4574,7 @@ mod tests {
                     },
                 ],
                 source: "".to_string(),
+                arrow_ffi_safe: false,
             })),
         };
 
diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto
index 3a1a5ca006..4afc1fefb7 100644
--- a/native/proto/src/proto/operator.proto
+++ b/native/proto/src/proto/operator.proto
@@ -81,8 +81,8 @@ message Scan {
   // is purely for informational purposes when viewing native query plans in
   // debug mode.
   string source = 2;
-  // field 3 was arrow_ffi_safe (removed - all batches are now ffi safe)
-  reserved 3;
+  // Whether native code can assume ownership of batches that it receives
+  bool arrow_ffi_safe = 3;
 }
 
 // Common data shared by all partitions in split mode (sent once at planning)
diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
index 4a8ae4d2ac..69b9bd5f85 100644
--- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala
@@ -96,6 +96,7 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec
       val scanOp = OperatorOuterClass.Scan
         .newBuilder()
         .setSource(cmd.query.nodeName)
+        .setArrowFfiSafe(false)
 
       // Add fields from the query output schema
       val scanTypes = cmd.query.output.flatMap { attr =>
diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
index f573fcd061..ca9dbdad7c 100644
--- a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala
@@ -36,6 +36,9 @@ import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataTy
  */
 abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
 
+  /** Whether the data produced by the Comet operator is FFI safe */
+  def isFfiSafe: Boolean = false
+
   override def enabledConfig: Option[ConfigEntry[Boolean]] = None
 
   override def convert(
@@ -58,6 +61,8 @@ abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
     } else {
       scanBuilder.setSource(source)
     }
+    scanBuilder.setArrowFfiSafe(isFfiSafe)
+
     val scanTypes = op.output.flatten { attr =>
       serializeDataType(attr.dataType)
     }
@@ -81,6 +86,15 @@ abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] {
 
 object CometExchangeSink extends CometSink[SparkPlan] {
 
+  /**
+   * Exchange data is FFI safe because there is no use of mutable buffers involved.
+   *
+   * Source of broadcast exchange batches is ArrowStreamReader.
+   *
+   * Source of shuffle exchange batches is NativeBatchDecoderIterator.
+   */
+  override def isFfiSafe: Boolean = true
+
   override def createExec(nativeOp: Operator, op: SparkPlan): CometNativeExec =
     CometSinkPlaceHolder(nativeOp, op, op)
 }
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
index 8012b18b22..4a323e575a 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
@@ -279,6 +279,13 @@ case class CometBroadcastExchangeExec(
 
 object CometBroadcastExchangeExec extends CometSink[BroadcastExchangeExec] {
 
+  /**
+   * Exchange data is FFI safe because there is no use of mutable buffers involved.
+   *
+   * Source of broadcast exchange batches is ArrowStreamReader.
+   */
+  override def isFfiSafe: Boolean = true
+
   override def enabledConfig: Option[ConfigEntry[Boolean]] = Some(
     CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED)
 
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
index 5928b76e6e..b804fe347e 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala
@@ -120,6 +120,7 @@ object CometLocalTableScanExec extends CometOperatorSerde[LocalTableScanExec] {
       .newBuilder()
       .setSource(op.getClass.getSimpleName)
       .addAllFields(scanTypes.asJava)
+      .setArrowFfiSafe(false)
     Some(builder.setScan(scanBuilder).build())
   }
 

From a9a6d5cbfe4e7f21cc2d0e63f35f476327ba3dec Mon Sep 17 00:00:00 2001
From: Andy Grove 
Date: Thu, 19 Mar 2026 18:33:53 -0600
Subject: [PATCH 7/7] Revert

---
 .../rules/EliminateRedundantTransitions.scala | 26 +++++++++++++++++--
 1 file changed, 24 insertions(+), 2 deletions(-)

diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
index 60ca2fe3ab..ce57624b75 100644
--- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
+++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala
@@ -22,7 +22,7 @@ package org.apache.comet.rules
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.util.sideBySide
-import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometSparkToColumnarExec}
+import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometScanExec, CometSparkToColumnarExec}
 import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec}
 import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan}
 import org.apache.spark.sql.execution.adaptive.QueryStageExec
@@ -139,7 +139,8 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa
   private def createColumnarToRowExec(child: SparkPlan): SparkPlan = {
     val schema = child.schema
     val useNative = CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.get() &&
-      CometNativeColumnarToRowExec.supportsSchema(schema)
+      CometNativeColumnarToRowExec.supportsSchema(schema) &&
+      !hasScanUsingMutableBuffers(child)
 
     if (useNative) {
       CometNativeColumnarToRowExec(child)
@@ -148,4 +149,25 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa
     }
   }
 
+  /**
+   * Checks if the plan contains a scan that uses mutable buffers. Native C2R is not compatible
+   * with such scans because the buffers may be modified after C2R reads them.
+   *
+   * This includes:
+   *   - CometScanExec with native_iceberg_compat and partition columns - uses
+   *     ConstantColumnReader
+   */
+  private def hasScanUsingMutableBuffers(op: SparkPlan): Boolean = {
+    op match {
+      case c: QueryStageExec => hasScanUsingMutableBuffers(c.plan)
+      case c: ReusedExchangeExec => hasScanUsingMutableBuffers(c.child)
+      case _ =>
+        op.exists {
+          case scan: CometScanExec =>
+            scan.scanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT &&
+            scan.relation.partitionSchema.nonEmpty
+          case _ => false
+        }
+    }
+  }
 }