To run this benchmark for spark-4.0:
+ * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh
+ * -PjmhIncludeRegex=TaskGroupPlanningBenchmark
+ * -PjmhOutputPath=benchmark/iceberg-task-group-planning-benchmark.txt
+ *
+ */
+@Fork(1)
+@State(Scope.Benchmark)
+@Warmup(iterations = 3)
+@Measurement(iterations = 5)
+@Timeout(time = 30, timeUnit = TimeUnit.MINUTES)
+@BenchmarkMode(Mode.SingleShotTime)
+public class TaskGroupPlanningBenchmark {
+
+ private static final String TABLE_NAME = "test_table";
+ private static final String PARTITION_COLUMN = "ss_ticket_number";
+
+ private static final int NUM_PARTITIONS = 150;
+ private static final int NUM_DATA_FILES_PER_PARTITION = 50_000;
+ private static final int NUM_DELETE_FILES_PER_PARTITION = 25;
+
+ private final Configuration hadoopConf = new Configuration();
+ private SparkSession spark;
+ private Table table;
+
+ private List fileTasks;
+
+ @Setup
+ public void setupBenchmark() throws NoSuchTableException, ParseException {
+ setupSpark();
+ initTable();
+ initDataAndDeletes();
+ loadFileTasks();
+ }
+
+ @TearDown
+ public void tearDownBenchmark() {
+ dropTable();
+ tearDownSpark();
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void planTaskGroups(Blackhole blackhole) {
+ SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of());
+ List> taskGroups =
+ TableScanUtil.planTaskGroups(
+ fileTasks,
+ readConf.splitSize(),
+ readConf.splitLookback(),
+ readConf.splitOpenFileCost());
+
+ long rowsCount = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ rowsCount += taskGroup.estimatedRowsCount();
+ }
+ blackhole.consume(rowsCount);
+
+ long filesCount = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ filesCount += taskGroup.filesCount();
+ }
+ blackhole.consume(filesCount);
+
+ long sizeBytes = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ sizeBytes += taskGroup.sizeBytes();
+ }
+ blackhole.consume(sizeBytes);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void planTaskGroupsWithGrouping(Blackhole blackhole) {
+ SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of());
+
+ List> taskGroups =
+ TableScanUtil.planTaskGroups(
+ fileTasks,
+ readConf.splitSize(),
+ readConf.splitLookback(),
+ readConf.splitOpenFileCost(),
+ Partitioning.groupingKeyType(table.schema(), table.specs().values()));
+
+ long rowsCount = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ rowsCount += taskGroup.estimatedRowsCount();
+ }
+ blackhole.consume(rowsCount);
+
+ long filesCount = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ filesCount += taskGroup.filesCount();
+ }
+ blackhole.consume(filesCount);
+
+ long sizeBytes = 0L;
+ for (ScanTaskGroup taskGroup : taskGroups) {
+ sizeBytes += taskGroup.sizeBytes();
+ }
+ blackhole.consume(sizeBytes);
+ }
+
+ private void loadFileTasks() {
+ table.refresh();
+
+ try (CloseableIterable fileTasksIterable = table.newScan().planFiles()) {
+ this.fileTasks = Lists.newArrayList(fileTasksIterable);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ private void initDataAndDeletes() {
+ for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) {
+ StructLike partition = TestHelpers.Row.of(partitionOrdinal);
+
+ RowDelta rowDelta = table.newRowDelta();
+
+ for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) {
+ DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition);
+ rowDelta.addRows(dataFile);
+ }
+
+ for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) {
+ DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition);
+ rowDelta.addDeletes(deleteFile);
+ }
+
+ rowDelta.commit();
+ }
+ }
+
+ private void setupSpark() {
+ this.spark =
+ SparkSession.builder()
+ .config("spark.ui.enabled", false)
+ .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName())
+ .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName())
+ .config("spark.sql.catalog.spark_catalog.type", "hadoop")
+ .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir())
+ .master("local[*]")
+ .getOrCreate();
+ }
+
+ private void tearDownSpark() {
+ spark.stop();
+ }
+
+ private void initTable() throws NoSuchTableException, ParseException {
+ sql(
+ "CREATE TABLE %s ( "
+ + " `ss_sold_date_sk` INT, "
+ + " `ss_sold_time_sk` INT, "
+ + " `ss_item_sk` INT, "
+ + " `ss_customer_sk` STRING, "
+ + " `ss_cdemo_sk` STRING, "
+ + " `ss_hdemo_sk` STRING, "
+ + " `ss_addr_sk` STRING, "
+ + " `ss_store_sk` STRING, "
+ + " `ss_promo_sk` STRING, "
+ + " `ss_ticket_number` INT, "
+ + " `ss_quantity` STRING, "
+ + " `ss_wholesale_cost` STRING, "
+ + " `ss_list_price` STRING, "
+ + " `ss_sales_price` STRING, "
+ + " `ss_ext_discount_amt` STRING, "
+ + " `ss_ext_sales_price` STRING, "
+ + " `ss_ext_wholesale_cost` STRING, "
+ + " `ss_ext_list_price` STRING, "
+ + " `ss_ext_tax` STRING, "
+ + " `ss_coupon_amt` STRING, "
+ + " `ss_net_paid` STRING, "
+ + " `ss_net_paid_inc_tax` STRING, "
+ + " `ss_net_profit` STRING "
+ + ")"
+ + "USING iceberg "
+ + "PARTITIONED BY (%s) "
+ + "TBLPROPERTIES ("
+ + " '%s' '%b',"
+ + " '%s' '%s',"
+ + " '%s' '%d')",
+ TABLE_NAME,
+ PARTITION_COLUMN,
+ TableProperties.MANIFEST_MERGE_ENABLED,
+ false,
+ TableProperties.DELETE_MODE,
+ RowLevelOperationMode.MERGE_ON_READ.modeName(),
+ TableProperties.FORMAT_VERSION,
+ 2);
+
+ this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME);
+ }
+
+ private void dropTable() {
+ sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME);
+ }
+
+ private String newWarehouseDir() {
+ return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID();
+ }
+
+ @FormatMethod
+ private void sql(@FormatString String query, Object... args) {
+ spark.sql(String.format(query, args));
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java
new file mode 100644
index 000000000000..d917eae5eb0f
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java
@@ -0,0 +1,213 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.spark.sql.functions.current_date;
+import static org.apache.spark.sql.functions.date_add;
+import static org.apache.spark.sql.functions.expr;
+
+import com.google.errorprone.annotations.FormatMethod;
+import com.google.errorprone.annotations.FormatString;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.parser.ParseException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+@Fork(1)
+@State(Scope.Benchmark)
+@Warmup(iterations = 3)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.SingleShotTime)
+public class UpdateProjectionBenchmark {
+
+ private static final String TABLE_NAME = "test_table";
+ private static final int NUM_FILES = 5;
+ private static final int NUM_ROWS_PER_FILE = 1_000_000;
+
+ private final Configuration hadoopConf = new Configuration();
+ private SparkSession spark;
+ private long originalSnapshotId;
+
+ @Setup
+ public void setupBenchmark() throws NoSuchTableException, ParseException {
+ setupSpark();
+ initTable();
+ appendData();
+
+ Table table = Spark3Util.loadIcebergTable(spark, TABLE_NAME);
+ this.originalSnapshotId = table.currentSnapshot().snapshotId();
+ }
+
+ @TearDown
+ public void tearDownBenchmark() {
+ tearDownSpark();
+ dropTable();
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void copyOnWriteUpdate10Percent() {
+ runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.1);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void copyOnWriteUpdate30Percent() {
+ runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.3);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void copyOnWriteUpdate75Percent() {
+ runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.75);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void mergeOnRead10Percent() {
+ runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.1);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void mergeOnReadUpdate30Percent() {
+ runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.3);
+ }
+
+ @Benchmark
+ @Threads(1)
+ public void mergeOnReadUpdate75Percent() {
+ runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.75);
+ }
+
+ private void runBenchmark(RowLevelOperationMode mode, double updatePercentage) {
+ sql(
+ "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')",
+ TABLE_NAME, TableProperties.UPDATE_MODE, mode.modeName());
+
+ int mod = (int) (NUM_ROWS_PER_FILE / (NUM_ROWS_PER_FILE * updatePercentage));
+
+ sql(
+ "UPDATE %s "
+ + "SET intCol = intCol + 10, dateCol = date_add(dateCol, 1) "
+ + "WHERE mod(id, %d) = 0",
+ TABLE_NAME, mod);
+
+ sql(
+ "CALL system.rollback_to_snapshot(table => '%s', snapshot_id => %dL)",
+ TABLE_NAME, originalSnapshotId);
+ }
+
+ private void setupSpark() {
+ this.spark =
+ SparkSession.builder()
+ .config("spark.ui.enabled", false)
+ .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName())
+ .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName())
+ .config("spark.sql.catalog.spark_catalog.type", "hadoop")
+ .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir())
+ .config(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false")
+ .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false")
+ .config(SQLConf.SHUFFLE_PARTITIONS().key(), "2")
+ .master("local")
+ .getOrCreate();
+ }
+
+ private void tearDownSpark() {
+ spark.stop();
+ }
+
+ private void initTable() {
+ sql(
+ "CREATE TABLE %s ( "
+ + " id LONG, intCol INT, floatCol FLOAT, doubleCol DOUBLE, "
+ + " decimalCol DECIMAL(20, 5), dateCol DATE, timestampCol TIMESTAMP, "
+ + " stringCol STRING)"
+ + "USING iceberg "
+ + "TBLPROPERTIES ("
+ + " '%s' '%s',"
+ + " '%s' '%d',"
+ + " '%s' '%d')",
+ TABLE_NAME,
+ TableProperties.UPDATE_DISTRIBUTION_MODE,
+ DistributionMode.NONE.modeName(),
+ TableProperties.SPLIT_OPEN_FILE_COST,
+ Integer.MAX_VALUE,
+ TableProperties.FORMAT_VERSION,
+ 2);
+
+ sql("ALTER TABLE %s WRITE ORDERED BY id", TABLE_NAME);
+ }
+
+ private void dropTable() {
+ sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME);
+ }
+
+ private void appendData() throws NoSuchTableException {
+ for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) {
+ Dataset inputDF =
+ spark
+ .range(NUM_ROWS_PER_FILE)
+ .withColumn("intCol", expr("CAST(id AS INT)"))
+ .withColumn("floatCol", expr("CAST(id AS FLOAT)"))
+ .withColumn("doubleCol", expr("CAST(id AS DOUBLE)"))
+ .withColumn("decimalCol", expr("CAST(id AS DECIMAL(20, 5))"))
+ .withColumn("dateCol", date_add(current_date(), fileNum))
+ .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)"))
+ .withColumn("stringCol", expr("CAST(dateCol AS STRING)"));
+ appendAsFile(inputDF);
+ }
+ }
+
+ private void appendAsFile(Dataset df) throws NoSuchTableException {
+ // ensure the schema is precise (including nullability)
+ StructType sparkSchema = spark.table(TABLE_NAME).schema();
+ spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(TABLE_NAME).append();
+ }
+
+ private String newWarehouseDir() {
+ return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID();
+ }
+
+ @FormatMethod
+ private void sql(@FormatString String query, Object... args) {
+ spark.sql(String.format(query, args));
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 b/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4
new file mode 100644
index 000000000000..b962699d9b47
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4
@@ -0,0 +1,374 @@
+/*
+ * 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.
+ *
+ * This file is an adaptation of Presto's and Spark's grammar files.
+ */
+
+grammar IcebergSqlExtensions;
+
+@lexer::members {
+ /**
+ * Verify whether current token is a valid decimal token (which contains dot).
+ * Returns true if the character that follows the token is not a digit or letter or underscore.
+ *
+ * For example:
+ * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'.
+ * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'.
+ * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'.
+ * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is followed
+ * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+'
+ * which is not a digit or letter or underscore.
+ */
+ public boolean isValidDecimal() {
+ int nextChar = _input.LA(1);
+ if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' ||
+ nextChar == '_') {
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ /**
+ * This method will be called when we see '/*' and try to match it as a bracketed comment.
+ * If the next character is '+', it should be parsed as hint later, and we cannot match
+ * it as a bracketed comment.
+ *
+ * Returns true if the next character is '+'.
+ */
+ public boolean isHint() {
+ int nextChar = _input.LA(1);
+ if (nextChar == '+') {
+ return true;
+ } else {
+ return false;
+ }
+ }
+}
+
+singleStatement
+ : statement EOF
+ ;
+
+statement
+ : CALL multipartIdentifier '(' (callArgument (',' callArgument)*)? ')' #call
+ | ALTER TABLE multipartIdentifier ADD PARTITION FIELD transform (AS name=identifier)? #addPartitionField
+ | ALTER TABLE multipartIdentifier DROP PARTITION FIELD transform #dropPartitionField
+ | ALTER TABLE multipartIdentifier REPLACE PARTITION FIELD transform WITH transform (AS name=identifier)? #replacePartitionField
+ | ALTER TABLE multipartIdentifier WRITE writeSpec #setWriteDistributionAndOrdering
+ | ALTER TABLE multipartIdentifier SET IDENTIFIER_KW FIELDS fieldList #setIdentifierFields
+ | ALTER TABLE multipartIdentifier DROP IDENTIFIER_KW FIELDS fieldList #dropIdentifierFields
+ | ALTER TABLE multipartIdentifier createReplaceBranchClause #createOrReplaceBranch
+ | ALTER TABLE multipartIdentifier createReplaceTagClause #createOrReplaceTag
+ | ALTER TABLE multipartIdentifier DROP BRANCH (IF EXISTS)? identifier #dropBranch
+ | ALTER TABLE multipartIdentifier DROP TAG (IF EXISTS)? identifier #dropTag
+ ;
+
+createReplaceTagClause
+ : (CREATE OR)? REPLACE TAG identifier tagOptions
+ | CREATE TAG (IF NOT EXISTS)? identifier tagOptions
+ ;
+
+createReplaceBranchClause
+ : (CREATE OR)? REPLACE BRANCH identifier branchOptions
+ | CREATE BRANCH (IF NOT EXISTS)? identifier branchOptions
+ ;
+
+tagOptions
+ : (AS OF VERSION snapshotId)? (refRetain)?
+ ;
+
+branchOptions
+ : (AS OF VERSION snapshotId)? (refRetain)? (snapshotRetention)?
+ ;
+
+snapshotRetention
+ : WITH SNAPSHOT RETENTION minSnapshotsToKeep
+ | WITH SNAPSHOT RETENTION maxSnapshotAge
+ | WITH SNAPSHOT RETENTION minSnapshotsToKeep maxSnapshotAge
+ ;
+
+refRetain
+ : RETAIN number timeUnit
+ ;
+
+maxSnapshotAge
+ : number timeUnit
+ ;
+
+minSnapshotsToKeep
+ : number SNAPSHOTS
+ ;
+
+writeSpec
+ : (writeDistributionSpec | writeOrderingSpec)*
+ ;
+
+writeDistributionSpec
+ : DISTRIBUTED BY PARTITION
+ ;
+
+writeOrderingSpec
+ : LOCALLY? ORDERED BY order
+ | UNORDERED
+ ;
+
+callArgument
+ : expression #positionalArgument
+ | identifier '=>' expression #namedArgument
+ ;
+
+singleOrder
+ : order EOF
+ ;
+
+order
+ : fields+=orderField (',' fields+=orderField)*
+ | '(' fields+=orderField (',' fields+=orderField)* ')'
+ ;
+
+orderField
+ : transform direction=(ASC | DESC)? (NULLS nullOrder=(FIRST | LAST))?
+ ;
+
+transform
+ : multipartIdentifier #identityTransform
+ | transformName=identifier
+ '(' arguments+=transformArgument (',' arguments+=transformArgument)* ')' #applyTransform
+ ;
+
+transformArgument
+ : multipartIdentifier
+ | constant
+ ;
+
+expression
+ : constant
+ | stringMap
+ | stringArray
+ ;
+
+constant
+ : number #numericLiteral
+ | booleanValue #booleanLiteral
+ | STRING+ #stringLiteral
+ | identifier STRING #typeConstructor
+ ;
+
+stringMap
+ : MAP '(' constant (',' constant)* ')'
+ ;
+
+stringArray
+ : ARRAY '(' constant (',' constant)* ')'
+ ;
+
+booleanValue
+ : TRUE | FALSE
+ ;
+
+number
+ : MINUS? EXPONENT_VALUE #exponentLiteral
+ | MINUS? DECIMAL_VALUE #decimalLiteral
+ | MINUS? INTEGER_VALUE #integerLiteral
+ | MINUS? BIGINT_LITERAL #bigIntLiteral
+ | MINUS? SMALLINT_LITERAL #smallIntLiteral
+ | MINUS? TINYINT_LITERAL #tinyIntLiteral
+ | MINUS? DOUBLE_LITERAL #doubleLiteral
+ | MINUS? FLOAT_LITERAL #floatLiteral
+ | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral
+ ;
+
+multipartIdentifier
+ : parts+=identifier ('.' parts+=identifier)*
+ ;
+
+identifier
+ : IDENTIFIER #unquotedIdentifier
+ | quotedIdentifier #quotedIdentifierAlternative
+ | nonReserved #unquotedIdentifier
+ ;
+
+quotedIdentifier
+ : BACKQUOTED_IDENTIFIER
+ ;
+
+fieldList
+ : fields+=multipartIdentifier (',' fields+=multipartIdentifier)*
+ ;
+
+nonReserved
+ : ADD | ALTER | AS | ASC | BRANCH | BY | CALL | CREATE | DAYS | DESC | DROP | EXISTS | FIELD | FIRST | HOURS | IF | LAST | NOT | NULLS | OF | OR | ORDERED | PARTITION | TABLE | WRITE
+ | DISTRIBUTED | LOCALLY | MINUTES | MONTHS | UNORDERED | REPLACE | RETAIN | VERSION | WITH | IDENTIFIER_KW | FIELDS | SET | SNAPSHOT | SNAPSHOTS
+ | TAG | TRUE | FALSE
+ | MAP
+ ;
+
+snapshotId
+ : number
+ ;
+
+numSnapshots
+ : number
+ ;
+
+timeUnit
+ : DAYS
+ | HOURS
+ | MINUTES
+ ;
+
+ADD: 'ADD';
+ALTER: 'ALTER';
+AS: 'AS';
+ASC: 'ASC';
+BRANCH: 'BRANCH';
+BY: 'BY';
+CALL: 'CALL';
+DAYS: 'DAYS';
+DESC: 'DESC';
+DISTRIBUTED: 'DISTRIBUTED';
+DROP: 'DROP';
+EXISTS: 'EXISTS';
+FIELD: 'FIELD';
+FIELDS: 'FIELDS';
+FIRST: 'FIRST';
+HOURS: 'HOURS';
+IF : 'IF';
+LAST: 'LAST';
+LOCALLY: 'LOCALLY';
+MINUTES: 'MINUTES';
+MONTHS: 'MONTHS';
+CREATE: 'CREATE';
+NOT: 'NOT';
+NULLS: 'NULLS';
+OF: 'OF';
+OR: 'OR';
+ORDERED: 'ORDERED';
+PARTITION: 'PARTITION';
+REPLACE: 'REPLACE';
+RETAIN: 'RETAIN';
+RETENTION: 'RETENTION';
+IDENTIFIER_KW: 'IDENTIFIER';
+SET: 'SET';
+SNAPSHOT: 'SNAPSHOT';
+SNAPSHOTS: 'SNAPSHOTS';
+TABLE: 'TABLE';
+TAG: 'TAG';
+UNORDERED: 'UNORDERED';
+VERSION: 'VERSION';
+WITH: 'WITH';
+WRITE: 'WRITE';
+
+TRUE: 'TRUE';
+FALSE: 'FALSE';
+
+MAP: 'MAP';
+ARRAY: 'ARRAY';
+
+PLUS: '+';
+MINUS: '-';
+
+STRING
+ : '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
+ | '"' ( ~('"'|'\\') | ('\\' .) )* '"'
+ ;
+
+BIGINT_LITERAL
+ : DIGIT+ 'L'
+ ;
+
+SMALLINT_LITERAL
+ : DIGIT+ 'S'
+ ;
+
+TINYINT_LITERAL
+ : DIGIT+ 'Y'
+ ;
+
+INTEGER_VALUE
+ : DIGIT+
+ ;
+
+EXPONENT_VALUE
+ : DIGIT+ EXPONENT
+ | DECIMAL_DIGITS EXPONENT {isValidDecimal()}?
+ ;
+
+DECIMAL_VALUE
+ : DECIMAL_DIGITS {isValidDecimal()}?
+ ;
+
+FLOAT_LITERAL
+ : DIGIT+ EXPONENT? 'F'
+ | DECIMAL_DIGITS EXPONENT? 'F' {isValidDecimal()}?
+ ;
+
+DOUBLE_LITERAL
+ : DIGIT+ EXPONENT? 'D'
+ | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}?
+ ;
+
+BIGDECIMAL_LITERAL
+ : DIGIT+ EXPONENT? 'BD'
+ | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}?
+ ;
+
+IDENTIFIER
+ : (LETTER | DIGIT | '_')+
+ ;
+
+BACKQUOTED_IDENTIFIER
+ : '`' ( ~'`' | '``' )* '`'
+ ;
+
+fragment DECIMAL_DIGITS
+ : DIGIT+ '.' DIGIT*
+ | '.' DIGIT+
+ ;
+
+fragment EXPONENT
+ : 'E' [+-]? DIGIT+
+ ;
+
+fragment DIGIT
+ : [0-9]
+ ;
+
+fragment LETTER
+ : [A-Z]
+ ;
+
+SIMPLE_COMMENT
+ : '--' ('\\\n' | ~[\r\n])* '\r'? '\n'? -> channel(HIDDEN)
+ ;
+
+BRACKETED_COMMENT
+ : '/*' {!isHint()}? (BRACKETED_COMMENT|.)*? '*/' -> channel(HIDDEN)
+ ;
+
+WS
+ : [ \r\n\t]+ -> channel(HIDDEN)
+ ;
+
+// Catch-all for anything we can't recognize.
+// We use this to be able to ignore and recover all the text
+// when splitting statements with DelimiterLexer
+UNRECOGNIZED
+ : .
+ ;
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
new file mode 100644
index 000000000000..3fca29c294c0
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.iceberg.spark.extensions
+
+import org.apache.spark.sql.SparkSessionExtensions
+import org.apache.spark.sql.catalyst.analysis.CheckViews
+import org.apache.spark.sql.catalyst.analysis.ProcedureArgumentCoercion
+import org.apache.spark.sql.catalyst.analysis.ResolveProcedures
+import org.apache.spark.sql.catalyst.analysis.ResolveViews
+import org.apache.spark.sql.catalyst.optimizer.ReplaceStaticInvoke
+import org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser
+import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Strategy
+
+class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
+
+ override def apply(extensions: SparkSessionExtensions): Unit = {
+ // parser extensions
+ extensions.injectParser { case (_, parser) => new IcebergSparkSqlExtensionsParser(parser) }
+
+ // analyzer extensions
+ extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
+ extensions.injectResolutionRule { spark => ResolveViews(spark) }
+ extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+ extensions.injectCheckRule(_ => CheckViews)
+
+ // optimizer extensions
+ extensions.injectOptimizerRule { _ => ReplaceStaticInvoke }
+
+ // planner extensions
+ extensions.injectPlannerStrategy { spark => ExtendedDataSourceV2Strategy(spark) }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala
new file mode 100644
index 000000000000..b559004b9466
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
+import org.apache.spark.sql.catalyst.plans.logical.View
+import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.SchemaUtils
+
+object CheckViews extends (LogicalPlan => Unit) {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override def apply(plan: LogicalPlan): Unit = {
+ plan foreach {
+ case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _,
+ _, _, _, _, replace, _) =>
+ verifyColumnCount(resolvedIdent, columnAliases, query)
+ SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver)
+ if (replace) {
+ val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier
+ checkCyclicViewReference(viewIdent, query, Seq(viewIdent))
+ }
+
+ case AlterViewAs(ResolvedV2View(_, _), _, _) =>
+ throw new IcebergAnalysisException(
+ "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead")
+ case _ => // OK
+ }
+ }
+
+ private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = {
+ if (columns.nonEmpty) {
+ if (columns.length > query.output.length) {
+ throw new AnalysisException(
+ errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
+ messageParameters = Map(
+ "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier),
+ "viewColumns" -> columns.mkString(", "),
+ "dataColumns" -> query.output.map(c => c.name).mkString(", ")))
+ } else if (columns.length < query.output.length) {
+ throw new AnalysisException(
+ errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS",
+ messageParameters = Map(
+ "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier),
+ "viewColumns" -> columns.mkString(", "),
+ "dataColumns" -> query.output.map(c => c.name).mkString(", ")))
+ }
+ }
+ }
+
+ private def checkCyclicViewReference(
+ viewIdent: Seq[String],
+ plan: LogicalPlan,
+ cyclePath: Seq[Seq[String]]): Unit = {
+ plan match {
+ case sub@SubqueryAlias(_, Project(_, _)) =>
+ val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name
+ checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children)
+ case v1View: View =>
+ val currentViewIdent: Seq[String] = v1View.desc.identifier.nameParts
+ checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, v1View.children)
+ case _ =>
+ plan.children.foreach(child => checkCyclicViewReference(viewIdent, child, cyclePath))
+ }
+
+ plan.expressions.flatMap(_.flatMap {
+ case e: SubqueryExpression =>
+ checkCyclicViewReference(viewIdent, e.plan, cyclePath)
+ None
+ case _ => None
+ })
+ }
+
+ private def checkIfRecursiveView(
+ viewIdent: Seq[String],
+ currentViewIdent: Seq[String],
+ cyclePath: Seq[Seq[String]],
+ children: Seq[LogicalPlan]
+ ): Unit = {
+ val newCyclePath = cyclePath :+ currentViewIdent
+ if (currentViewIdent == viewIdent) {
+ throw new IcebergAnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)",
+ viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> ")))
+ } else {
+ children.foreach { c =>
+ checkCyclicViewReference(viewIdent, c, newCyclePath)
+ }
+ }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala
new file mode 100644
index 000000000000..01dbd6952618
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.IcebergCall
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+object ProcedureArgumentCoercion extends Rule[LogicalPlan] {
+ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+ case c @ IcebergCall(procedure, args) if c.resolved =>
+ val params = procedure.parameters
+
+ val newArgs = args.zipWithIndex.map { case (arg, index) =>
+ val param = params(index)
+ val paramType = param.dataType
+ val argType = arg.dataType
+
+ if (paramType != argType && !Cast.canUpCast(argType, paramType)) {
+ throw new IcebergAnalysisException(
+ s"Wrong arg type for ${param.name}: cannot cast $argType to $paramType")
+ }
+
+ if (paramType != argType) {
+ Cast(arg, paramType)
+ } else {
+ arg
+ }
+ }
+
+ if (newArgs != args) {
+ c.copy(args = newArgs)
+ } else {
+ c
+ }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala
new file mode 100644
index 000000000000..2d02d4ce76e0
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import java.util.Locale
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.CallArgument
+import org.apache.spark.sql.catalyst.plans.logical.CallStatement
+import org.apache.spark.sql.catalyst.plans.logical.IcebergCall
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.NamedArgument
+import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.CatalogPlugin
+import org.apache.spark.sql.connector.catalog.LookupCatalog
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter
+import scala.collection.Seq
+
+case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog {
+
+ protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+ case CallStatement(CatalogAndIdentifier(catalog, ident), args) =>
+ val procedure = catalog.asProcedureCatalog.loadProcedure(ident)
+
+ val params = procedure.parameters
+ val normalizedParams = normalizeParams(params)
+ validateParams(normalizedParams)
+
+ val normalizedArgs = normalizeArgs(args)
+ IcebergCall(procedure, args = buildArgExprs(normalizedParams, normalizedArgs).toSeq)
+ }
+
+ private def validateParams(params: Seq[ProcedureParameter]): Unit = {
+ // should not be any duplicate param names
+ val duplicateParamNames = params.groupBy(_.name).collect {
+ case (name, matchingParams) if matchingParams.length > 1 => name
+ }
+
+ if (duplicateParamNames.nonEmpty) {
+ throw new IcebergAnalysisException(s"Duplicate parameter names: ${duplicateParamNames.mkString("[", ",", "]")}")
+ }
+
+ // optional params should be at the end
+ params.sliding(2).foreach {
+ case Seq(previousParam, currentParam) if !previousParam.required && currentParam.required =>
+ throw new IcebergAnalysisException(
+ s"Optional parameters must be after required ones but $currentParam is after $previousParam")
+ case _ =>
+ }
+ }
+
+ private def buildArgExprs(
+ params: Seq[ProcedureParameter],
+ args: Seq[CallArgument]): Seq[Expression] = {
+
+ // build a map of declared parameter names to their positions
+ val nameToPositionMap = params.map(_.name).zipWithIndex.toMap
+
+ // build a map of parameter names to args
+ val nameToArgMap = buildNameToArgMap(params, args, nameToPositionMap)
+
+ // verify all required parameters are provided
+ val missingParamNames = params.filter(_.required).collect {
+ case param if !nameToArgMap.contains(param.name) => param.name
+ }
+
+ if (missingParamNames.nonEmpty) {
+ throw new IcebergAnalysisException(s"Missing required parameters: ${missingParamNames.mkString("[", ",", "]")}")
+ }
+
+ val argExprs = new Array[Expression](params.size)
+
+ nameToArgMap.foreach { case (name, arg) =>
+ val position = nameToPositionMap(name)
+ argExprs(position) = arg.expr
+ }
+
+ // assign nulls to optional params that were not set
+ params.foreach {
+ case p if !p.required && !nameToArgMap.contains(p.name) =>
+ val position = nameToPositionMap(p.name)
+ argExprs(position) = Literal.create(null, p.dataType)
+ case _ =>
+ }
+
+ argExprs
+ }
+
+ private def buildNameToArgMap(
+ params: Seq[ProcedureParameter],
+ args: Seq[CallArgument],
+ nameToPositionMap: Map[String, Int]): Map[String, CallArgument] = {
+
+ val containsNamedArg = args.exists(_.isInstanceOf[NamedArgument])
+ val containsPositionalArg = args.exists(_.isInstanceOf[PositionalArgument])
+
+ if (containsNamedArg && containsPositionalArg) {
+ throw new IcebergAnalysisException("Named and positional arguments cannot be mixed")
+ }
+
+ if (containsNamedArg) {
+ buildNameToArgMapUsingNames(args, nameToPositionMap)
+ } else {
+ buildNameToArgMapUsingPositions(args, params)
+ }
+ }
+
+ private def buildNameToArgMapUsingNames(
+ args: Seq[CallArgument],
+ nameToPositionMap: Map[String, Int]): Map[String, CallArgument] = {
+
+ val namedArgs = args.asInstanceOf[Seq[NamedArgument]]
+
+ val validationErrors = namedArgs.groupBy(_.name).collect {
+ case (name, matchingArgs) if matchingArgs.size > 1 => s"Duplicate procedure argument: $name"
+ case (name, _) if !nameToPositionMap.contains(name) => s"Unknown argument: $name"
+ }
+
+ if (validationErrors.nonEmpty) {
+ throw new IcebergAnalysisException(s"Could not build name to arg map: ${validationErrors.mkString(", ")}")
+ }
+
+ namedArgs.map(arg => arg.name -> arg).toMap
+ }
+
+ private def buildNameToArgMapUsingPositions(
+ args: Seq[CallArgument],
+ params: Seq[ProcedureParameter]): Map[String, CallArgument] = {
+
+ if (args.size > params.size) {
+ throw new IcebergAnalysisException("Too many arguments for procedure")
+ }
+
+ args.zipWithIndex.map { case (arg, position) =>
+ val param = params(position)
+ param.name -> arg
+ }.toMap
+ }
+
+ private def normalizeParams(params: Seq[ProcedureParameter]): Seq[ProcedureParameter] = {
+ params.map {
+ case param if param.required =>
+ val normalizedName = param.name.toLowerCase(Locale.ROOT)
+ ProcedureParameter.required(normalizedName, param.dataType)
+ case param =>
+ val normalizedName = param.name.toLowerCase(Locale.ROOT)
+ ProcedureParameter.optional(normalizedName, param.dataType)
+ }
+ }
+
+ private def normalizeArgs(args: Seq[CallArgument]): Seq[CallArgument] = {
+ args.map {
+ case a @ NamedArgument(name, _) => a.copy(name = name.toLowerCase(Locale.ROOT))
+ case other => other
+ }
+ }
+
+ implicit class CatalogHelper(plugin: CatalogPlugin) {
+ def asProcedureCatalog: ProcedureCatalog = plugin match {
+ case procedureCatalog: ProcedureCatalog =>
+ procedureCatalog
+ case _ =>
+ throw new IcebergAnalysisException(s"Cannot use catalog ${plugin.name}: not a ProcedureCatalog")
+ }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala
new file mode 100644
index 000000000000..397b70b188d4
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.ViewUtil.IcebergViewHelper
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.expressions.UpCast
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
+import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.CurrentOrigin
+import org.apache.spark.sql.catalyst.trees.Origin
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.LookupCatalog
+import org.apache.spark.sql.connector.catalog.View
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.MetadataBuilder
+
+case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+ case u@UnresolvedRelation(nameParts, _, _)
+ if catalogManager.v1SessionCatalog.isTempView(nameParts) =>
+ u
+
+ case u@UnresolvedRelation(parts@CatalogAndIdentifier(catalog, ident), _, _) =>
+ ViewUtil.loadView(catalog, ident)
+ .map(createViewRelation(parts, _))
+ .getOrElse(u)
+
+ case u@UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) =>
+ ViewUtil.loadView(catalog, ident)
+ .map(_ => ResolvedV2View(catalog.asViewCatalog, ident))
+ .getOrElse(u)
+
+ case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _)
+ if query.resolved && !c.rewritten =>
+ val aliased = aliasColumns(query, columnAliases, columnComments)
+ c.copy(query = aliased, queryColumnNames = query.schema.fieldNames, rewritten = true)
+ }
+
+ private def aliasColumns(
+ plan: LogicalPlan,
+ columnAliases: Seq[String],
+ columnComments: Seq[Option[String]]): LogicalPlan = {
+ if (columnAliases.isEmpty || columnAliases.length != plan.output.length) {
+ plan
+ } else {
+ val projectList = plan.output.zipWithIndex.map { case (attr, pos) =>
+ if (columnComments.apply(pos).isDefined) {
+ val meta = new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build()
+ Alias(attr, columnAliases.apply(pos))(explicitMetadata = Some(meta))
+ } else {
+ Alias(attr, columnAliases.apply(pos))()
+ }
+ }
+ Project(projectList, plan)
+ }
+ }
+
+
+ private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = {
+ val parsed = parseViewText(nameParts.quoted, view.query)
+
+ // Apply any necessary rewrites to preserve correct resolution
+ val viewCatalogAndNamespace: Seq[String] = view.currentCatalog +: view.currentNamespace.toSeq
+ val rewritten = rewriteIdentifiers(parsed, viewCatalogAndNamespace);
+
+ // Apply the field aliases and column comments
+ // This logic differs from how Spark handles views in SessionCatalog.fromCatalogTable.
+ // This is more strict because it doesn't allow resolution by field name.
+ val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) =>
+ val attr = GetColumnByOrdinal(pos, expected.dataType)
+ Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata))
+ }
+
+ SubqueryAlias(nameParts, Project(aliases, rewritten))
+ }
+
+ private def parseViewText(name: String, viewText: String): LogicalPlan = {
+ val origin = Origin(
+ objectType = Some("VIEW"),
+ objectName = Some(name)
+ )
+
+ try {
+ CurrentOrigin.withOrigin(origin) {
+ spark.sessionState.sqlParser.parseQuery(viewText)
+ }
+ } catch {
+ case _: ParseException =>
+ throw QueryCompilationErrors.invalidViewNameError(name);
+ }
+ }
+
+ private def rewriteIdentifiers(
+ plan: LogicalPlan,
+ catalogAndNamespace: Seq[String]): LogicalPlan = {
+ // Substitute CTEs within the view, then rewrite unresolved functions and relations
+ qualifyTableIdentifiers(
+ qualifyFunctionIdentifiers(
+ CTESubstitution.apply(plan),
+ catalogAndNamespace),
+ catalogAndNamespace)
+ }
+
+ private def qualifyFunctionIdentifiers(
+ plan: LogicalPlan,
+ catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions {
+ case u@UnresolvedFunction(Seq(name), _, _, _, _, _, _) =>
+ if (!isBuiltinFunction(name)) {
+ u.copy(nameParts = catalogAndNamespace :+ name)
+ } else {
+ u
+ }
+ case u@UnresolvedFunction(parts, _, _, _, _, _, _) if !isCatalog(parts.head) =>
+ u.copy(nameParts = catalogAndNamespace.head +: parts)
+ }
+
+ /**
+ * Qualify table identifiers with default catalog and namespace if necessary.
+ */
+ private def qualifyTableIdentifiers(
+ child: LogicalPlan,
+ catalogAndNamespace: Seq[String]): LogicalPlan =
+ child transform {
+ case u@UnresolvedRelation(Seq(table), _, _) =>
+ u.copy(multipartIdentifier = catalogAndNamespace :+ table)
+ case u@UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) =>
+ u.copy(multipartIdentifier = catalogAndNamespace.head +: parts)
+ case other =>
+ other.transformExpressions {
+ case subquery: SubqueryExpression =>
+ subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace))
+ }
+ }
+
+ private def isCatalog(name: String): Boolean = {
+ catalogManager.isCatalogRegistered(name)
+ }
+
+ private def isBuiltinFunction(name: String): Boolean = {
+ catalogManager.v1SessionCatalog.isBuiltinFunction(FunctionIdentifier(name))
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
new file mode 100644
index 000000000000..0546da1653d4
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.ViewUtil.IcebergViewHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.plans.logical.CreateView
+import org.apache.spark.sql.catalyst.plans.logical.DropView
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ShowViews
+import org.apache.spark.sql.catalyst.plans.logical.View
+import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View
+import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.LookupCatalog
+import scala.collection.mutable
+
+/**
+ * ResolveSessionCatalog exits early for some v2 View commands,
+ * thus they are pre-substituted here and then handled in ResolveViews
+ */
+case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+ case DropView(ResolvedIdent(resolved), ifExists) =>
+ DropIcebergView(resolved, ifExists)
+
+ case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, properties,
+ Some(queryText), query, allowExisting, replace, _) =>
+ val q = CTESubstitution.apply(query)
+ verifyTemporaryObjectsDontExist(resolved, q)
+ CreateIcebergView(child = resolved,
+ queryText = queryText,
+ query = q,
+ columnAliases = userSpecifiedColumns.map(_._1),
+ columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)),
+ comment = comment,
+ properties = properties,
+ allowExisting = allowExisting,
+ replace = replace)
+
+ case view @ ShowViews(CurrentNamespace, pattern, output) =>
+ if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) {
+ ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace),
+ pattern, output)
+ } else {
+ view
+ }
+
+ case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns), _), pattern, output)
+ if ViewUtil.isViewCatalog(catalog) =>
+ ShowIcebergViews(ResolvedNamespace(catalog, ns), pattern, output)
+
+ // needs to be done here instead of in ResolveViews, so that a V2 view can be resolved before the Analyzer
+ // tries to resolve it, which would result in an error, saying that V2 views aren't supported
+ case u@UnresolvedView(ResolvedView(resolved), _, _, _) =>
+ ViewUtil.loadView(resolved.catalog, resolved.identifier)
+ .map(_ => ResolvedV2View(resolved.catalog.asViewCatalog, resolved.identifier))
+ .getOrElse(u)
+ }
+
+ private def isTempView(nameParts: Seq[String]): Boolean = {
+ catalogManager.v1SessionCatalog.isTempView(nameParts)
+ }
+
+ private def isTempFunction(nameParts: Seq[String]): Boolean = {
+ if (nameParts.size > 1) {
+ return false
+ }
+ catalogManager.v1SessionCatalog.isTemporaryFunction(nameParts.asFunctionIdentifier)
+ }
+
+ private object ResolvedIdent {
+ def unapply(unresolved: UnresolvedIdentifier): Option[ResolvedIdentifier] = unresolved match {
+ case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) =>
+ None
+
+ case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) if ViewUtil.isViewCatalog(catalog) =>
+ Some(ResolvedIdentifier(catalog, ident))
+
+ case _ =>
+ None
+ }
+ }
+
+ /**
+ * Permanent views are not allowed to reference temp objects
+ */
+ private def verifyTemporaryObjectsDontExist(
+ identifier: ResolvedIdentifier,
+ child: LogicalPlan): Unit = {
+ val tempViews = collectTemporaryViews(child)
+ if (tempViews.nonEmpty) {
+ throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view")
+ }
+
+ val tempFunctions = collectTemporaryFunctions(child)
+ if (tempFunctions.nonEmpty) {
+ throw invalidRefToTempObject(identifier, tempFunctions.mkString("[", ", ", "]"), "function")
+ }
+ }
+
+ private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = {
+ new IcebergAnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s",
+ ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames))
+ }
+
+ /**
+ * Collect all temporary views and return the identifiers separately
+ */
+ private def collectTemporaryViews(child: LogicalPlan): Seq[Seq[String]] = {
+ def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = {
+ child.flatMap {
+ case unresolved: UnresolvedRelation if isTempView(unresolved.multipartIdentifier) =>
+ Seq(unresolved.multipartIdentifier)
+ case view: View if view.isTempView => Seq(view.desc.identifier.nameParts)
+ case plan => plan.expressions.flatMap(_.flatMap {
+ case e: SubqueryExpression => collectTempViews(e.plan)
+ case _ => Seq.empty
+ })
+ }.distinct
+ }
+
+ collectTempViews(child)
+ }
+
+ private object ResolvedView {
+ def unapply(identifier: Seq[String]): Option[ResolvedV2View] = identifier match {
+ case nameParts if isTempView(nameParts) =>
+ None
+
+ case CatalogAndIdentifier(catalog, ident) if ViewUtil.isViewCatalog(catalog) =>
+ ViewUtil.loadView(catalog, ident).flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident)))
+
+ case _ =>
+ None
+ }
+ }
+
+ /**
+ * Collect the names of all temporary functions.
+ */
+ private def collectTemporaryFunctions(child: LogicalPlan): Seq[String] = {
+ val tempFunctions = new mutable.HashSet[String]()
+ child.resolveExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) {
+ case f @ UnresolvedFunction(nameParts, _, _, _, _, _, _) if isTempFunction(nameParts) =>
+ tempFunctions += nameParts.head
+ f
+ case e: SubqueryExpression =>
+ tempFunctions ++= collectTemporaryFunctions(e.plan)
+ e
+ }
+ tempFunctions.toSeq
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala
new file mode 100644
index 000000000000..d46f10b7f5a2
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.connector.catalog.CatalogPlugin
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.View
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+object ViewUtil {
+ def loadView(catalog: CatalogPlugin, ident: Identifier): Option[View] = catalog match {
+ case viewCatalog: ViewCatalog =>
+ try {
+ Option(viewCatalog.loadView(ident))
+ } catch {
+ case _: NoSuchViewException => None
+ }
+ case _ => None
+ }
+
+ def isViewCatalog(catalog: CatalogPlugin): Boolean = {
+ catalog.isInstanceOf[ViewCatalog]
+ }
+
+ implicit class IcebergViewHelper(plugin: CatalogPlugin) {
+ def asViewCatalog: ViewCatalog = plugin match {
+ case viewCatalog: ViewCatalog =>
+ viewCatalog
+ case _ =>
+ throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "views")
+ }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala
new file mode 100644
index 000000000000..d5c4cb84a02a
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.iceberg.spark.functions.SparkFunctions
+import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression
+import org.apache.spark.sql.catalyst.expressions.BinaryComparison
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.In
+import org.apache.spark.sql.catalyst.expressions.InSet
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.BINARY_COMPARISON
+import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND
+import org.apache.spark.sql.catalyst.trees.TreePattern.FILTER
+import org.apache.spark.sql.catalyst.trees.TreePattern.IN
+import org.apache.spark.sql.catalyst.trees.TreePattern.INSET
+import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Spark analyzes the Iceberg system function to {@link StaticInvoke} which could not be pushed
+ * down to datasource. This rule will replace {@link StaticInvoke} to
+ * {@link ApplyFunctionExpression} for Iceberg system function in a filter condition.
+ */
+object ReplaceStaticInvoke extends Rule[LogicalPlan] {
+
+ override def apply(plan: LogicalPlan): LogicalPlan =
+ plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) {
+ case replace @ ReplaceData(_, cond, _, _, _, _) =>
+ replaceStaticInvoke(replace, cond, newCond => replace.copy(condition = newCond))
+
+ case join @ Join(_, _, _, Some(cond), _) =>
+ replaceStaticInvoke(join, cond, newCond => join.copy(condition = Some(newCond)))
+
+ case filter @ Filter(cond, _) =>
+ replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond))
+ }
+
+ private def replaceStaticInvoke[T <: LogicalPlan](
+ node: T,
+ condition: Expression,
+ copy: Expression => T): T = {
+ val newCondition = replaceStaticInvoke(condition)
+ if (newCondition fastEquals condition) node else copy(newCondition)
+ }
+
+ private def replaceStaticInvoke(condition: Expression): Expression = {
+ condition.transformWithPruning(_.containsAnyPattern(BINARY_COMPARISON, IN, INSET)) {
+ case in @ In(value: StaticInvoke, _) if canReplace(value) =>
+ in.copy(value = replaceStaticInvoke(value))
+
+ case in @ InSet(value: StaticInvoke, _) if canReplace(value) =>
+ in.copy(child = replaceStaticInvoke(value))
+
+ case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable =>
+ c.withNewChildren(Seq(replaceStaticInvoke(left), right))
+
+ case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable =>
+ c.withNewChildren(Seq(left, replaceStaticInvoke(right)))
+ }
+ }
+
+ private def replaceStaticInvoke(invoke: StaticInvoke): Expression = {
+ // Adaptive from `resolveV2Function` in org.apache.spark.sql.catalyst.analysis.ResolveFunctions
+ val unbound = SparkFunctions.loadFunctionByClass(invoke.staticObject)
+ if (unbound == null) {
+ return invoke
+ }
+
+ val inputType = StructType(invoke.arguments.zipWithIndex.map {
+ case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+ })
+
+ val bound = try {
+ unbound.bind(inputType)
+ } catch {
+ case _: Exception =>
+ return invoke
+ }
+
+ if (bound.inputTypes().length != invoke.arguments.length) {
+ return invoke
+ }
+
+ bound match {
+ case scalarFunc: ScalarFunction[_] =>
+ ApplyFunctionExpression(scalarFunc, invoke.arguments)
+ case _ => invoke
+ }
+ }
+
+ @inline
+ private def canReplace(invoke: StaticInvoke): Boolean = {
+ invoke.functionName == ScalarFunction.MAGIC_METHOD_NAME && !invoke.foldable
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
new file mode 100644
index 000000000000..554a06d9610b
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
@@ -0,0 +1,346 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser.extensions
+
+import java.util.Locale
+import org.antlr.v4.runtime._
+import org.antlr.v4.runtime.atn.PredictionMode
+import org.antlr.v4.runtime.misc.Interval
+import org.antlr.v4.runtime.misc.ParseCancellationException
+import org.antlr.v4.runtime.tree.TerminalNodeImpl
+import org.apache.iceberg.common.DynConstructors
+import org.apache.iceberg.spark.ExtendedParser
+import org.apache.iceberg.spark.ExtendedParser.RawOrderField
+import org.apache.iceberg.spark.procedures.SparkProcedures
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.RewriteViewCommands
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.parser.CompoundBody
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser.NonReservedContext
+import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser.QuotedIdentifierContext
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.Origin
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.VariableSubstitution
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.StructType
+import scala.jdk.CollectionConverters._
+
+class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface with ExtendedParser {
+
+ import IcebergSparkSqlExtensionsParser._
+
+ private lazy val substitutor = substitutorCtor.newInstance(SQLConf.get)
+ private lazy val astBuilder = new IcebergSqlExtensionsAstBuilder(delegate)
+
+ /**
+ * Parse a string to a DataType.
+ */
+ override def parseDataType(sqlText: String): DataType = {
+ delegate.parseDataType(sqlText)
+ }
+
+ /**
+ * Parse a string to a raw DataType without CHAR/VARCHAR replacement.
+ */
+ def parseRawDataType(sqlText: String): DataType = throw new UnsupportedOperationException()
+
+ /**
+ * Parse a string to an Expression.
+ */
+ override def parseExpression(sqlText: String): Expression = {
+ delegate.parseExpression(sqlText)
+ }
+
+ /**
+ * Parse a string to a TableIdentifier.
+ */
+ override def parseTableIdentifier(sqlText: String): TableIdentifier = {
+ delegate.parseTableIdentifier(sqlText)
+ }
+
+ /**
+ * Parse a string to a FunctionIdentifier.
+ */
+ override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = {
+ delegate.parseFunctionIdentifier(sqlText)
+ }
+
+ /**
+ * Parse a string to a multi-part identifier.
+ */
+ override def parseMultipartIdentifier(sqlText: String): Seq[String] = {
+ delegate.parseMultipartIdentifier(sqlText)
+ }
+
+ /**
+ * Creates StructType for a given SQL string, which is a comma separated list of field
+ * definitions which will preserve the correct Hive metadata.
+ */
+ override def parseTableSchema(sqlText: String): StructType = {
+ delegate.parseTableSchema(sqlText)
+ }
+
+ override def parseScript(sqlScriptText: String): CompoundBody = {
+ delegate.parseScript(sqlScriptText)
+ }
+
+ override def parseSortOrder(sqlText: String): java.util.List[RawOrderField] = {
+ val fields = parse(sqlText) { parser => astBuilder.visitSingleOrder(parser.singleOrder()) }
+ fields.map { field =>
+ val (term, direction, order) = field
+ new RawOrderField(term, direction, order)
+ }.asJava
+ }
+
+ /**
+ * Parse a string to a LogicalPlan.
+ */
+ override def parsePlan(sqlText: String): LogicalPlan = {
+ val sqlTextAfterSubstitution = substitutor.substitute(sqlText)
+ if (isIcebergCommand(sqlTextAfterSubstitution)) {
+ parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan]
+ } else {
+ RewriteViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText))
+ }
+ }
+
+ private def isIcebergCommand(sqlText: String): Boolean = {
+ val normalized = sqlText.toLowerCase(Locale.ROOT).trim()
+ // Strip simple SQL comments that terminate a line, e.g. comments starting with `--` .
+ .replaceAll("--.*?\\n", " ")
+ // Strip newlines.
+ .replaceAll("\\s+", " ")
+ // Strip comments of the form /* ... */. This must come after stripping newlines so that
+ // comments that span multiple lines are caught.
+ .replaceAll("/\\*.*?\\*/", " ")
+ // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of
+ .replaceAll("`", "")
+ .trim()
+
+ isIcebergProcedure(normalized) || (
+ normalized.startsWith("alter table") && (
+ normalized.contains("add partition field") ||
+ normalized.contains("drop partition field") ||
+ normalized.contains("replace partition field") ||
+ normalized.contains("write ordered by") ||
+ normalized.contains("write locally ordered by") ||
+ normalized.contains("write distributed by") ||
+ normalized.contains("write unordered") ||
+ normalized.contains("set identifier fields") ||
+ normalized.contains("drop identifier fields") ||
+ isSnapshotRefDdl(normalized)))
+ }
+
+ // All builtin Iceberg procedures are under the 'system' namespace
+ private def isIcebergProcedure(normalized: String): Boolean = {
+ normalized.startsWith("call") &&
+ SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains)
+ }
+
+ private def isSnapshotRefDdl(normalized: String): Boolean = {
+ normalized.contains("create branch") ||
+ normalized.contains("replace branch") ||
+ normalized.contains("create tag") ||
+ normalized.contains("replace tag") ||
+ normalized.contains("drop branch") ||
+ normalized.contains("drop tag")
+ }
+
+ protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = {
+ val lexer = new IcebergSqlExtensionsLexer(new UpperCaseCharStream(CharStreams.fromString(command)))
+ lexer.removeErrorListeners()
+ lexer.addErrorListener(IcebergParseErrorListener)
+
+ val tokenStream = new CommonTokenStream(lexer)
+ val parser = new IcebergSqlExtensionsParser(tokenStream)
+ parser.addParseListener(IcebergSqlExtensionsPostProcessor)
+ parser.removeErrorListeners()
+ parser.addErrorListener(IcebergParseErrorListener)
+
+ try {
+ try {
+ // first, try parsing with potentially faster SLL mode
+ parser.getInterpreter.setPredictionMode(PredictionMode.SLL)
+ toResult(parser)
+ }
+ catch {
+ case _: ParseCancellationException =>
+ // if we fail, parse with LL mode
+ tokenStream.seek(0) // rewind input stream
+ parser.reset()
+
+ // Try Again.
+ parser.getInterpreter.setPredictionMode(PredictionMode.LL)
+ toResult(parser)
+ }
+ }
+ catch {
+ case e: IcebergParseException if e.command.isDefined =>
+ throw e
+ case e: IcebergParseException =>
+ throw e.withCommand(command)
+ case e: AnalysisException =>
+ val position = Origin(e.line, e.startPosition)
+ throw new IcebergParseException(Option(command), e.message, position, position)
+ }
+ }
+
+ override def parseQuery(sqlText: String): LogicalPlan = {
+ parsePlan(sqlText)
+ }
+}
+
+object IcebergSparkSqlExtensionsParser {
+ private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] =
+ DynConstructors.builder()
+ .impl(classOf[VariableSubstitution])
+ .impl(classOf[VariableSubstitution], classOf[SQLConf])
+ .build()
+}
+
+/* Copied from Apache Spark's to avoid dependency on Spark Internals */
+class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream {
+ override def consume(): Unit = wrapped.consume
+ override def getSourceName(): String = wrapped.getSourceName
+ override def index(): Int = wrapped.index
+ override def mark(): Int = wrapped.mark
+ override def release(marker: Int): Unit = wrapped.release(marker)
+ override def seek(where: Int): Unit = wrapped.seek(where)
+ override def size(): Int = wrapped.size
+
+ override def getText(interval: Interval): String = wrapped.getText(interval)
+
+ // scalastyle:off
+ override def LA(i: Int): Int = {
+ val la = wrapped.LA(i)
+ if (la == 0 || la == IntStream.EOF) la
+ else Character.toUpperCase(la)
+ }
+ // scalastyle:on
+}
+
+/**
+ * The post-processor validates & cleans-up the parse tree during the parse process.
+ */
+case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseListener {
+
+ /** Remove the back ticks from an Identifier. */
+ override def exitQuotedIdentifier(ctx: QuotedIdentifierContext): Unit = {
+ replaceTokenByIdentifier(ctx, 1) { token =>
+ // Remove the double back ticks in the string.
+ token.setText(token.getText.replace("``", "`"))
+ token
+ }
+ }
+
+ /** Treat non-reserved keywords as Identifiers. */
+ override def exitNonReserved(ctx: NonReservedContext): Unit = {
+ replaceTokenByIdentifier(ctx, 0)(identity)
+ }
+
+ private def replaceTokenByIdentifier(
+ ctx: ParserRuleContext,
+ stripMargins: Int)(
+ f: CommonToken => CommonToken = identity): Unit = {
+ val parent = ctx.getParent
+ parent.removeLastChild()
+ val token = ctx.getChild(0).getPayload.asInstanceOf[Token]
+ val newToken = new CommonToken(
+ new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream),
+ IcebergSqlExtensionsParser.IDENTIFIER,
+ token.getChannel,
+ token.getStartIndex + stripMargins,
+ token.getStopIndex - stripMargins)
+ parent.addChild(new TerminalNodeImpl(f(newToken)))
+ }
+}
+
+/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */
+case object IcebergParseErrorListener extends BaseErrorListener {
+ override def syntaxError(
+ recognizer: Recognizer[_, _],
+ offendingSymbol: scala.Any,
+ line: Int,
+ charPositionInLine: Int,
+ msg: String,
+ e: RecognitionException): Unit = {
+ val (start, stop) = offendingSymbol match {
+ case token: CommonToken =>
+ val start = Origin(Some(line), Some(token.getCharPositionInLine))
+ val length = token.getStopIndex - token.getStartIndex + 1
+ val stop = Origin(Some(line), Some(token.getCharPositionInLine + length))
+ (start, stop)
+ case _ =>
+ val start = Origin(Some(line), Some(charPositionInLine))
+ (start, start)
+ }
+ throw new IcebergParseException(None, msg, start, stop)
+ }
+}
+
+/**
+ * Copied from Apache Spark
+ * A [[ParseException]] is an [[AnalysisException]] that is thrown during the parse process. It
+ * contains fields and an extended error message that make reporting and diagnosing errors easier.
+ */
+class IcebergParseException(
+ val command: Option[String],
+ message: String,
+ val start: Origin,
+ val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) {
+
+ def this(message: String, ctx: ParserRuleContext) = {
+ this(Option(IcebergParserUtils.command(ctx)),
+ message,
+ IcebergParserUtils.position(ctx.getStart),
+ IcebergParserUtils.position(ctx.getStop))
+ }
+
+ override def getMessage: String = {
+ val builder = new StringBuilder
+ builder ++= "\n" ++= message
+ start match {
+ case Origin(Some(l), Some(p), Some(startIndex), Some(stopIndex), Some(sqlText),
+ Some(objectType), Some(objectName), _, _) =>
+ builder ++= s"(line $l, pos $p)\n"
+ command.foreach { cmd =>
+ val (above, below) = cmd.split("\n").splitAt(l)
+ builder ++= "\n== SQL ==\n"
+ above.foreach(builder ++= _ += '\n')
+ builder ++= (0 until p).map(_ => "-").mkString("") ++= "^^^\n"
+ below.foreach(builder ++= _ += '\n')
+ }
+ case _ =>
+ command.foreach { cmd =>
+ builder ++= "\n== SQL ==\n" ++= cmd
+ }
+ }
+ builder.toString
+ }
+
+ def withCommand(cmd: String): IcebergParseException = {
+ new IcebergParseException(Option(cmd), message, start, stop)
+ }
+}
\ No newline at end of file
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala
new file mode 100644
index 000000000000..b95fc7755fb9
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser.extensions
+
+import java.util.Locale
+import java.util.concurrent.TimeUnit
+import org.antlr.v4.runtime._
+import org.antlr.v4.runtime.misc.Interval
+import org.antlr.v4.runtime.tree.ParseTree
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.iceberg.DistributionMode
+import org.apache.iceberg.NullOrder
+import org.apache.iceberg.SortDirection
+import org.apache.iceberg.expressions.Term
+import org.apache.iceberg.spark.Spark3Util
+import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.parser.extensions.IcebergParserUtils.withOrigin
+import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser._
+import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField
+import org.apache.spark.sql.catalyst.plans.logical.BranchOptions
+import org.apache.spark.sql.catalyst.plans.logical.CallArgument
+import org.apache.spark.sql.catalyst.plans.logical.CallStatement
+import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch
+import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag
+import org.apache.spark.sql.catalyst.plans.logical.DropBranch
+import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields
+import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField
+import org.apache.spark.sql.catalyst.plans.logical.DropTag
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.NamedArgument
+import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument
+import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField
+import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields
+import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering
+import org.apache.spark.sql.catalyst.plans.logical.TagOptions
+import org.apache.spark.sql.catalyst.trees.CurrentOrigin
+import org.apache.spark.sql.catalyst.trees.Origin
+import org.apache.spark.sql.connector.expressions
+import org.apache.spark.sql.connector.expressions.ApplyTransform
+import org.apache.spark.sql.connector.expressions.FieldReference
+import org.apache.spark.sql.connector.expressions.IdentityTransform
+import org.apache.spark.sql.connector.expressions.LiteralValue
+import org.apache.spark.sql.connector.expressions.Transform
+import scala.jdk.CollectionConverters._
+
+class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergSqlExtensionsBaseVisitor[AnyRef] {
+
+ private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = list.asScala
+ private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq
+
+ /**
+ * Create a [[CallStatement]] for a stored procedure call.
+ */
+ override def visitCall(ctx: CallContext): CallStatement = withOrigin(ctx) {
+ val name = toSeq(ctx.multipartIdentifier.parts).map(_.getText)
+ val args = toSeq(ctx.callArgument).map(typedVisit[CallArgument])
+ CallStatement(name, args)
+ }
+
+ /**
+ * Create an ADD PARTITION FIELD logical command.
+ */
+ override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = withOrigin(ctx) {
+ AddPartitionField(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ typedVisit[Transform](ctx.transform),
+ Option(ctx.name).map(_.getText))
+ }
+
+ /**
+ * Create a DROP PARTITION FIELD logical command.
+ */
+ override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = withOrigin(ctx) {
+ DropPartitionField(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ typedVisit[Transform](ctx.transform))
+ }
+
+ /**
+ * Create a CREATE OR REPLACE BRANCH logical command.
+ */
+ override def visitCreateOrReplaceBranch(ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) {
+ val createOrReplaceBranchClause = ctx.createReplaceBranchClause()
+
+ val branchName = createOrReplaceBranchClause.identifier()
+ val branchOptionsContext = Option(createOrReplaceBranchClause.branchOptions())
+ val snapshotId = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotId()))
+ .map(_.getText.toLong)
+ val snapshotRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention()))
+ val minSnapshotsToKeep = snapshotRetention.flatMap(retention => Option(retention.minSnapshotsToKeep()))
+ .map(minSnapshots => minSnapshots.number().getText.toLong)
+ val maxSnapshotAgeMs = snapshotRetention
+ .flatMap(retention => Option(retention.maxSnapshotAge()))
+ .map(retention => TimeUnit.valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH))
+ .toMillis(retention.number().getText.toLong))
+ val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain()))
+ val branchRefAgeMs = branchRetention.map(retain =>
+ TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong))
+ val create = createOrReplaceBranchClause.CREATE() != null
+ val replace = ctx.createReplaceBranchClause().REPLACE() != null
+ val ifNotExists = createOrReplaceBranchClause.EXISTS() != null
+
+ val branchOptions = BranchOptions(
+ snapshotId,
+ minSnapshotsToKeep,
+ maxSnapshotAgeMs,
+ branchRefAgeMs
+ )
+
+ CreateOrReplaceBranch(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ branchName.getText,
+ branchOptions,
+ create,
+ replace,
+ ifNotExists)
+ }
+
+ /**
+ * Create an CREATE OR REPLACE TAG logical command.
+ */
+ override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = withOrigin(ctx) {
+ val createTagClause = ctx.createReplaceTagClause()
+
+ val tagName = createTagClause.identifier().getText
+
+ val tagOptionsContext = Option(createTagClause.tagOptions())
+ val snapshotId = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.snapshotId()))
+ .map(_.getText.toLong)
+ val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain()))
+ val tagRefAgeMs = tagRetain.map(retain =>
+ TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong))
+ val tagOptions = TagOptions(
+ snapshotId,
+ tagRefAgeMs
+ )
+
+ val create = createTagClause.CREATE() != null
+ val replace = createTagClause.REPLACE() != null
+ val ifNotExists = createTagClause.EXISTS() != null
+
+ CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier),
+ tagName,
+ tagOptions,
+ create,
+ replace,
+ ifNotExists)
+ }
+
+ /**
+ * Create an DROP BRANCH logical command.
+ */
+ override def visitDropBranch(ctx: DropBranchContext): DropBranch = withOrigin(ctx) {
+ DropBranch(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null)
+ }
+
+ /**
+ * Create an DROP TAG logical command.
+ */
+ override def visitDropTag(ctx: DropTagContext): DropTag = withOrigin(ctx) {
+ DropTag(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null)
+ }
+
+ /**
+ * Create an REPLACE PARTITION FIELD logical command.
+ */
+ override def visitReplacePartitionField(ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) {
+ ReplacePartitionField(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ typedVisit[Transform](ctx.transform(0)),
+ typedVisit[Transform](ctx.transform(1)),
+ Option(ctx.name).map(_.getText))
+ }
+
+ /**
+ * Create an SET IDENTIFIER FIELDS logical command.
+ */
+ override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = withOrigin(ctx) {
+ SetIdentifierFields(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ toSeq(ctx.fieldList.fields).map(_.getText))
+ }
+
+ /**
+ * Create an DROP IDENTIFIER FIELDS logical command.
+ */
+ override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) {
+ DropIdentifierFields(
+ typedVisit[Seq[String]](ctx.multipartIdentifier),
+ toSeq(ctx.fieldList.fields).map(_.getText))
+ }
+
+ /**
+ * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering.
+ */
+ override def visitSetWriteDistributionAndOrdering(
+ ctx: SetWriteDistributionAndOrderingContext): SetWriteDistributionAndOrdering = {
+
+ val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier)
+
+ val (distributionSpec, orderingSpec) = toDistributionAndOrderingSpec(ctx.writeSpec)
+
+ if (distributionSpec == null && orderingSpec == null) {
+ throw new IcebergAnalysisException(
+ "ALTER TABLE has no changes: missing both distribution and ordering clauses")
+ }
+
+ val distributionMode = if (distributionSpec != null) {
+ Some(DistributionMode.HASH)
+ } else if (orderingSpec.UNORDERED != null) {
+ Some(DistributionMode.NONE)
+ } else if (orderingSpec.LOCALLY() != null) {
+ None
+ } else {
+ Some(DistributionMode.RANGE)
+ }
+
+ val ordering = if (orderingSpec != null && orderingSpec.order != null) {
+ toSeq(orderingSpec.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)])
+ } else {
+ Seq.empty
+ }
+
+ SetWriteDistributionAndOrdering(tableName, distributionMode, ordering)
+ }
+
+ private def toDistributionAndOrderingSpec(
+ writeSpec: WriteSpecContext): (WriteDistributionSpecContext, WriteOrderingSpecContext) = {
+
+ if (writeSpec.writeDistributionSpec.size > 1) {
+ throw new IcebergAnalysisException("ALTER TABLE contains multiple distribution clauses")
+ }
+
+ if (writeSpec.writeOrderingSpec.size > 1) {
+ throw new IcebergAnalysisException("ALTER TABLE contains multiple ordering clauses")
+ }
+
+ val distributionSpec = toBuffer(writeSpec.writeDistributionSpec).headOption.orNull
+ val orderingSpec = toBuffer(writeSpec.writeOrderingSpec).headOption.orNull
+
+ (distributionSpec, orderingSpec)
+ }
+
+ /**
+ * Create an order field.
+ */
+ override def visitOrderField(ctx: OrderFieldContext): (Term, SortDirection, NullOrder) = {
+ val term = Spark3Util.toIcebergTerm(typedVisit[Transform](ctx.transform))
+ val direction = Option(ctx.ASC).map(_ => SortDirection.ASC)
+ .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC))
+ .getOrElse(SortDirection.ASC)
+ val nullOrder = Option(ctx.FIRST).map(_ => NullOrder.NULLS_FIRST)
+ .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST))
+ .getOrElse(if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST)
+ (term, direction, nullOrder)
+ }
+
+ /**
+ * Create an IdentityTransform for a column reference.
+ */
+ override def visitIdentityTransform(ctx: IdentityTransformContext): Transform = withOrigin(ctx) {
+ IdentityTransform(FieldReference(typedVisit[Seq[String]](ctx.multipartIdentifier())))
+ }
+
+ /**
+ * Create a named Transform from argument expressions.
+ */
+ override def visitApplyTransform(ctx: ApplyTransformContext): Transform = withOrigin(ctx) {
+ val args = toSeq(ctx.arguments).map(typedVisit[expressions.Expression])
+ ApplyTransform(ctx.transformName.getText, args)
+ }
+
+ /**
+ * Create a transform argument from a column reference or a constant.
+ */
+ override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = withOrigin(ctx) {
+ val reference = Option(ctx.multipartIdentifier())
+ .map(typedVisit[Seq[String]])
+ .map(FieldReference(_))
+ val literal = Option(ctx.constant)
+ .map(visitConstant)
+ .map(lit => LiteralValue(lit.value, lit.dataType))
+ reference.orElse(literal)
+ .getOrElse(throw new IcebergParseException(s"Invalid transform argument", ctx))
+ }
+
+ /**
+ * Return a multi-part identifier as Seq[String].
+ */
+ override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) {
+ toSeq(ctx.parts).map(_.getText)
+ }
+
+ override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = withOrigin(ctx) {
+ toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)])
+ }
+
+ /**
+ * Create a positional argument in a stored procedure call.
+ */
+ override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = withOrigin(ctx) {
+ val expr = typedVisit[Expression](ctx.expression)
+ PositionalArgument(expr)
+ }
+
+ /**
+ * Create a named argument in a stored procedure call.
+ */
+ override def visitNamedArgument(ctx: NamedArgumentContext): CallArgument = withOrigin(ctx) {
+ val name = ctx.identifier.getText
+ val expr = typedVisit[Expression](ctx.expression)
+ NamedArgument(name, expr)
+ }
+
+ override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
+ visit(ctx.statement).asInstanceOf[LogicalPlan]
+ }
+
+ def visitConstant(ctx: ConstantContext): Literal = {
+ delegate.parseExpression(ctx.getText).asInstanceOf[Literal]
+ }
+
+ override def visitExpression(ctx: ExpressionContext): Expression = {
+ // reconstruct the SQL string and parse it using the main Spark parser
+ // while we can avoid the logic to build Spark expressions, we still have to parse them
+ // we cannot call ctx.getText directly since it will not render spaces correctly
+ // that's why we need to recurse down the tree in reconstructSqlString
+ val sqlString = reconstructSqlString(ctx)
+ delegate.parseExpression(sqlString)
+ }
+
+ private def reconstructSqlString(ctx: ParserRuleContext): String = {
+ toBuffer(ctx.children).map {
+ case c: ParserRuleContext => reconstructSqlString(c)
+ case t: TerminalNode => t.getText
+ }.mkString(" ")
+ }
+
+ private def typedVisit[T](ctx: ParseTree): T = {
+ ctx.accept(this).asInstanceOf[T]
+ }
+}
+
+/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */
+object IcebergParserUtils {
+
+ private[sql] def withOrigin[T](ctx: ParserRuleContext)(f: => T): T = {
+ val current = CurrentOrigin.get
+ CurrentOrigin.set(position(ctx.getStart))
+ try {
+ f
+ } finally {
+ CurrentOrigin.set(current)
+ }
+ }
+
+ private[sql] def position(token: Token): Origin = {
+ val opt = Option(token)
+ Origin(opt.map(_.getLine), opt.map(_.getCharPositionInLine))
+ }
+
+ /** Get the command which created the token. */
+ private[sql] def command(ctx: ParserRuleContext): String = {
+ val stream = ctx.getStart.getInputStream
+ stream.getText(Interval.of(0, stream.size() - 1))
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala
new file mode 100644
index 000000000000..e8b1b2941161
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) extends LeafCommand {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"AddPartitionField ${table.quoted} ${name.map(n => s"$n=").getOrElse("")}${transform.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala
new file mode 100644
index 000000000000..4d7e0a086bda
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+case class BranchOptions (snapshotId: Option[Long], numSnapshots: Option[Long],
+ snapshotRetain: Option[Long], snapshotRefRetain: Option[Long])
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala
new file mode 100644
index 000000000000..9e3fdb0e9e0e
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.connector.iceberg.catalog.Procedure
+
+case class Call(procedure: Procedure, args: Seq[Expression]) extends LeafCommand {
+ override lazy val output: Seq[Attribute] = DataTypeUtils.toAttributes(procedure.outputType)
+
+ override def simpleString(maxFields: Int): String = {
+ s"Call${truncatedString(output.toSeq, "[", ", ", "]", maxFields)} ${procedure.description}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala
new file mode 100644
index 000000000000..b7981a3c7a0d
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+case class CreateOrReplaceBranch(
+ table: Seq[String],
+ branch: String,
+ branchOptions: BranchOptions,
+ create: Boolean,
+ replace: Boolean,
+ ifNotExists: Boolean) extends LeafCommand {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"CreateOrReplaceBranch branch: ${branch} for table: ${table.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala
new file mode 100644
index 000000000000..6e7db84a90fb
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+case class CreateOrReplaceTag(
+ table: Seq[String],
+ tag: String,
+ tagOptions: TagOptions,
+ create: Boolean,
+ replace: Boolean,
+ ifNotExists: Boolean) extends LeafCommand {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"CreateOrReplaceTag tag: ${tag} for table: ${table.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala
new file mode 100644
index 000000000000..bee0b0fae688
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+case class DropBranch(table: Seq[String], branch: String, ifExists: Boolean) extends LeafCommand {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropBranch branch: ${branch} for table: ${table.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala
new file mode 100644
index 000000000000..29dd686a0fba
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+case class DropIdentifierFields(
+ table: Seq[String],
+ fields: Seq[String]) extends LeafCommand {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropIdentifierFields ${table.quoted} (${fields.quoted})"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala
new file mode 100644
index 000000000000..fb1451324182
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class DropPartitionField(table: Seq[String], transform: Transform) extends LeafCommand {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropPartitionField ${table.quoted} ${transform.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala
new file mode 100644
index 000000000000..7e4b38e74d2f
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+case class DropTag(table: Seq[String], tag: String, ifExists: Boolean) extends LeafCommand {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropTag tag: ${tag} for table: ${table.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IcebergCall.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IcebergCall.scala
new file mode 100644
index 000000000000..032eac13970a
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IcebergCall.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.connector.iceberg.catalog.Procedure
+
+case class IcebergCall(procedure: Procedure, args: Seq[Expression]) extends LeafCommand {
+ override lazy val output: Seq[Attribute] = DataTypeUtils.toAttributes(procedure.outputType)
+
+ override def simpleString(maxFields: Int): String = {
+ s"IcebergCall${truncatedString(output.toSeq, "[", ", ", "]", maxFields)} ${procedure.description}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala
new file mode 100644
index 000000000000..8c660c6f37b1
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class ReplacePartitionField(
+ table: Seq[String],
+ transformFrom: Transform,
+ transformTo: Transform,
+ name: Option[String]) extends LeafCommand {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"ReplacePartitionField ${table.quoted} ${transformFrom.describe} " +
+ s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala
new file mode 100644
index 000000000000..a5fa28a617e7
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class SetIdentifierFields(
+ table: Seq[String],
+ fields: Seq[String]) extends LeafCommand {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override def simpleString(maxFields: Int): String = {
+ s"SetIdentifierFields ${table.quoted} (${fields.quoted})"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala
new file mode 100644
index 000000000000..85e3b95f4aba
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long])
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
new file mode 100644
index 000000000000..be15f32bc1b8
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+
+/**
+ * A CALL statement, as parsed from SQL.
+ */
+case class CallStatement(name: Seq[String], args: Seq[CallArgument]) extends LeafParsedStatement
+
+/**
+ * An argument in a CALL statement.
+ */
+sealed trait CallArgument {
+ def expr: Expression
+}
+
+/**
+ * An argument in a CALL statement identified by name.
+ */
+case class NamedArgument(name: String, expr: Expression) extends CallArgument
+
+/**
+ * An argument in a CALL statement identified by position.
+ */
+case class PositionalArgument(expr: Expression) extends CallArgument
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala
new file mode 100644
index 000000000000..9366d5efe163
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical.views
+
+import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+case class CreateIcebergView(
+ child: LogicalPlan,
+ queryText: String,
+ query: LogicalPlan,
+ columnAliases: Seq[String],
+ columnComments: Seq[Option[String]],
+ queryColumnNames: Seq[String] = Seq.empty,
+ comment: Option[String],
+ properties: Map[String, String],
+ allowExisting: Boolean,
+ replace: Boolean,
+ rewritten: Boolean = false) extends BinaryCommand {
+ override def left: LogicalPlan = child
+
+ override def right: LogicalPlan = query
+
+ override protected def withNewChildrenInternal(
+ newLeft: LogicalPlan, newRight: LogicalPlan): LogicalPlan =
+ copy(child = newLeft, query = newRight)
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala
new file mode 100644
index 000000000000..275dba6fbf5e
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical.views
+
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand
+
+case class DropIcebergView(
+ child: LogicalPlan,
+ ifExists: Boolean) extends UnaryCommand {
+ override protected def withNewChildInternal(newChild: LogicalPlan): DropIcebergView =
+ copy(child = newChild)
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala
new file mode 100644
index 000000000000..b9c05ff0061d
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical.views
+
+import org.apache.spark.sql.catalyst.analysis.LeafNodeWithoutStats
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+
+case class ResolvedV2View(
+ catalog: ViewCatalog,
+ identifier: Identifier) extends LeafNodeWithoutStats {
+ override def output: Seq[Attribute] = Nil
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala
new file mode 100644
index 000000000000..b09c27acdc16
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.spark.sql.catalyst.plans.logical.views
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ShowViews
+import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand
+
+case class ShowIcebergViews(
+ namespace: LogicalPlan,
+ pattern: Option[String],
+ override val output: Seq[Attribute] = ShowViews.getOutputAttrs) extends UnaryCommand {
+ override def child: LogicalPlan = namespace
+
+ override protected def withNewChildInternal(newChild: LogicalPlan): ShowIcebergViews =
+ copy(namespace = newChild)
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala
new file mode 100644
index 000000000000..55f327f7e45e
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.Spark3Util
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class AddPartitionFieldExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ transform: Transform,
+ name: Option[String]) extends LeafV2CommandExec {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ iceberg.table.updateSpec()
+ .addField(name.orNull, Spark3Util.toIcebergTerm(transform))
+ .commit()
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot add partition field to non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"AddPartitionField ${catalog.name}.${ident.quoted} ${name.map(n => s"$n=").getOrElse("")}${transform.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala
new file mode 100644
index 000000000000..b103d1ee2c58
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.connector.catalog.ViewChange
+
+
+case class AlterV2ViewSetPropertiesExec(
+ catalog: ViewCatalog,
+ ident: Identifier,
+ properties: Map[String, String]) extends LeafV2CommandExec {
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ val changes = properties.map {
+ case (property, value) => ViewChange.setProperty(property, value)
+ }.toSeq
+
+ catalog.alterView(ident, changes: _*)
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"AlterV2ViewSetProperties: ${ident}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala
new file mode 100644
index 000000000000..a4103fede24c
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.connector.catalog.ViewChange
+
+
+case class AlterV2ViewUnsetPropertiesExec(
+ catalog: ViewCatalog,
+ ident: Identifier,
+ propertyKeys: Seq[String],
+ ifExists: Boolean) extends LeafV2CommandExec {
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ if (!ifExists) {
+ propertyKeys.filterNot(catalog.loadView(ident).properties.containsKey).foreach { property =>
+ throw new IcebergAnalysisException(s"Cannot remove property that is not set: '$property'")
+ }
+ }
+
+ val changes = propertyKeys.map(ViewChange.removeProperty)
+ catalog.alterView(ident, changes: _*)
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"AlterV2ViewUnsetProperties: ${ident}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala
new file mode 100644
index 000000000000..f66962a8c453
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.connector.iceberg.catalog.Procedure
+import scala.collection.compat.immutable.ArraySeq
+
+case class CallExec(
+ output: Seq[Attribute],
+ procedure: Procedure,
+ input: InternalRow) extends LeafV2CommandExec {
+
+ override protected def run(): Seq[InternalRow] = {
+ ArraySeq.unsafeWrapArray(procedure.call(input))
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"CallExec${truncatedString(output, "[", ", ", "]", maxFields)} ${procedure.description}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala
new file mode 100644
index 000000000000..2be406e7f344
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.BranchOptions
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+case class CreateOrReplaceBranchExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ branch: String,
+ branchOptions: BranchOptions,
+ create: Boolean,
+ replace: Boolean,
+ ifNotExists: Boolean) extends LeafV2CommandExec {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val snapshotId: java.lang.Long = branchOptions.snapshotId
+ .orElse(Option(iceberg.table.currentSnapshot()).map(_.snapshotId()))
+ .map(java.lang.Long.valueOf)
+ .orNull
+
+ val manageSnapshots = iceberg.table().manageSnapshots()
+ val refExists = null != iceberg.table().refs().get(branch)
+
+ def safeCreateBranch(): Unit = {
+ if (snapshotId == null) {
+ manageSnapshots.createBranch(branch)
+ } else {
+ manageSnapshots.createBranch(branch, snapshotId)
+ }
+ }
+
+ if (create && replace && !refExists) {
+ safeCreateBranch()
+ } else if (replace) {
+ Preconditions.checkArgument(snapshotId != null,
+ "Cannot complete replace branch operation on %s, main has no snapshot", ident)
+ manageSnapshots.replaceBranch(branch, snapshotId)
+ } else {
+ if (refExists && ifNotExists) {
+ return Nil
+ }
+
+ safeCreateBranch()
+ }
+
+ if (branchOptions.numSnapshots.nonEmpty) {
+ manageSnapshots.setMinSnapshotsToKeep(branch, branchOptions.numSnapshots.get.toInt)
+ }
+
+ if (branchOptions.snapshotRetain.nonEmpty) {
+ manageSnapshots.setMaxSnapshotAgeMs(branch, branchOptions.snapshotRetain.get)
+ }
+
+ if (branchOptions.snapshotRefRetain.nonEmpty) {
+ manageSnapshots.setMaxRefAgeMs(branch, branchOptions.snapshotRefRetain.get)
+ }
+
+ manageSnapshots.commit()
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot create or replace branch on non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"CreateOrReplace branch: $branch for table: ${ident.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala
new file mode 100644
index 000000000000..372cd7548632
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.TagOptions
+import org.apache.spark.sql.connector.catalog._
+
+case class CreateOrReplaceTagExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ tag: String,
+ tagOptions: TagOptions,
+ create: Boolean,
+ replace: Boolean,
+ ifNotExists: Boolean) extends LeafV2CommandExec {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val snapshotId: java.lang.Long = tagOptions.snapshotId
+ .orElse(Option(iceberg.table.currentSnapshot()).map(_.snapshotId()))
+ .map(java.lang.Long.valueOf)
+ .orNull
+
+ Preconditions.checkArgument(snapshotId != null,
+ "Cannot complete create or replace tag operation on %s, main has no snapshot", ident)
+
+ val manageSnapshot = iceberg.table.manageSnapshots()
+ val refExists = null != iceberg.table().refs().get(tag)
+
+ if (create && replace && !refExists) {
+ manageSnapshot.createTag(tag, snapshotId)
+ } else if (replace) {
+ manageSnapshot.replaceTag(tag, snapshotId)
+ } else {
+ if (refExists && ifNotExists) {
+ return Nil
+ }
+
+ manageSnapshot.createTag(tag, snapshotId)
+ }
+
+ if (tagOptions.snapshotRefRetain.nonEmpty) {
+ manageSnapshot.setMaxRefAgeMs(tag, tagOptions.snapshotRefRetain.get)
+ }
+
+ manageSnapshot.commit()
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot create tag to non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"Create tag: $tag for table: ${ident.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala
new file mode 100644
index 000000000000..9015fb338ea5
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.SupportsReplaceView
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.NoSuchViewException
+import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.connector.catalog.ViewInfo
+import org.apache.spark.sql.types.StructType
+import scala.collection.JavaConverters._
+
+
+case class CreateV2ViewExec(
+ catalog: ViewCatalog,
+ ident: Identifier,
+ queryText: String,
+ viewSchema: StructType,
+ columnAliases: Seq[String],
+ columnComments: Seq[Option[String]],
+ queryColumnNames: Seq[String],
+ comment: Option[String],
+ properties: Map[String, String],
+ allowExisting: Boolean,
+ replace: Boolean) extends LeafV2CommandExec {
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ val currentCatalogName = session.sessionState.catalogManager.currentCatalog.name
+ val currentCatalog = if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null
+ val currentNamespace = session.sessionState.catalogManager.currentNamespace
+
+ val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION
+ val newProperties = properties ++
+ comment.map(ViewCatalog.PROP_COMMENT -> _) +
+ (ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion,
+ ViewCatalog.PROP_ENGINE_VERSION -> engineVersion)
+
+ if (replace) {
+ // CREATE OR REPLACE VIEW
+ catalog match {
+ case c: SupportsReplaceView =>
+ try {
+ replaceView(c, currentCatalog, currentNamespace, newProperties)
+ } catch {
+ // view might have been concurrently dropped during replace
+ case _: NoSuchViewException =>
+ replaceView(c, currentCatalog, currentNamespace, newProperties)
+ }
+ case _ =>
+ if (catalog.viewExists(ident)) {
+ catalog.dropView(ident)
+ }
+
+ createView(currentCatalog, currentNamespace, newProperties)
+ }
+ } else {
+ try {
+ // CREATE VIEW [IF NOT EXISTS]
+ createView(currentCatalog, currentNamespace, newProperties)
+ } catch {
+ case _: ViewAlreadyExistsException if allowExisting => // Ignore
+ }
+ }
+
+ Nil
+ }
+
+ private def replaceView(
+ supportsReplaceView: SupportsReplaceView,
+ currentCatalog: String,
+ currentNamespace: Array[String],
+ newProperties: Map[String, String]) = {
+ supportsReplaceView.replaceView(
+ ident,
+ queryText,
+ currentCatalog,
+ currentNamespace,
+ viewSchema,
+ queryColumnNames.toArray,
+ columnAliases.toArray,
+ columnComments.map(c => c.orNull).toArray,
+ newProperties.asJava)
+ }
+
+ private def createView(
+ currentCatalog: String,
+ currentNamespace: Array[String],
+ newProperties: Map[String, String]) = {
+ val viewInfo: ViewInfo = new ViewInfo(
+ ident,
+ queryText,
+ currentCatalog,
+ currentNamespace,
+ viewSchema,
+ queryColumnNames.toArray,
+ columnAliases.toArray,
+ columnComments.map(c => c.orNull).toArray,
+ newProperties.asJava)
+ catalog.createView(viewInfo)
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"CreateV2ViewExec: ${ident}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala
new file mode 100644
index 000000000000..bb08fb18b2bd
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString
+import org.apache.spark.sql.connector.catalog.View
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.execution.LeafExecNode
+import scala.collection.JavaConverters._
+
+case class DescribeV2ViewExec(
+ output: Seq[Attribute],
+ view: View,
+ isExtended: Boolean) extends V2CommandExec with LeafExecNode {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override protected def run(): Seq[InternalRow] = {
+ if (isExtended) {
+ (describeSchema :+ emptyRow) ++ describeExtended
+ } else {
+ describeSchema
+ }
+ }
+
+ private def describeSchema: Seq[InternalRow] =
+ view.schema().map { column =>
+ toCatalystRow(
+ column.name,
+ column.dataType.simpleString,
+ column.getComment().getOrElse(""))
+ }
+
+ private def emptyRow: InternalRow = toCatalystRow("", "", "")
+
+ private def describeExtended: Seq[InternalRow] = {
+ val outputColumns = view.queryColumnNames.mkString("[", ", ", "]")
+ val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala
+ val viewCatalogAndNamespace: Seq[String] = view.currentCatalog +: view.currentNamespace.toSeq
+ val viewProperties = properties.toSeq.sortBy(_._1).map {
+ case (key, value) =>
+ s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'"
+ }.mkString("[", ", ", "]")
+
+
+ toCatalystRow("# Detailed View Information", "", "") ::
+ toCatalystRow("Comment", view.properties.getOrDefault(ViewCatalog.PROP_COMMENT, ""), "") ::
+ toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") ::
+ toCatalystRow("View Query Output Columns", outputColumns, "") ::
+ toCatalystRow("View Properties", viewProperties, "") ::
+ toCatalystRow("Created By", view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), "") ::
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DescribeV2ViewExec"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala
new file mode 100644
index 000000000000..ff8f1820099a
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+case class DropBranchExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ branch: String,
+ ifExists: Boolean) extends LeafV2CommandExec {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val ref = iceberg.table().refs().get(branch)
+ if (ref != null || !ifExists) {
+ iceberg.table().manageSnapshots().removeBranch(branch).commit()
+ }
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot drop branch on non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropBranch branch: ${branch} for table: ${ident.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala
new file mode 100644
index 000000000000..dee778b474f9
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions
+import org.apache.iceberg.relocated.com.google.common.collect.Sets
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+case class DropIdentifierFieldsExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ fields: Seq[String]) extends LeafV2CommandExec {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val schema = iceberg.table.schema
+ val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames)
+
+ for (name <- fields) {
+ Preconditions.checkArgument(schema.findField(name) != null,
+ "Cannot complete drop identifier fields operation: field %s not found", name)
+ Preconditions.checkArgument(identifierFieldNames.contains(name),
+ "Cannot complete drop identifier fields operation: %s is not an identifier field", name)
+ identifierFieldNames.remove(name)
+ }
+
+ iceberg.table.updateSchema()
+ .setIdentifierFields(identifierFieldNames)
+ .commit();
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot drop identifier fields in non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})";
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala
new file mode 100644
index 000000000000..9a153f0c004e
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.Spark3Util
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+import org.apache.spark.sql.connector.expressions.FieldReference
+import org.apache.spark.sql.connector.expressions.IdentityTransform
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class DropPartitionFieldExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ transform: Transform) extends LeafV2CommandExec {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val schema = iceberg.table.schema
+ transform match {
+ case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null =>
+ // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name
+ iceberg.table.updateSpec()
+ .removeField(parts.head)
+ .commit()
+
+ case _ =>
+ iceberg.table.updateSpec()
+ .removeField(Spark3Util.toIcebergTerm(transform))
+ .commit()
+ }
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot drop partition field in non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropPartitionField ${catalog.name}.${ident.quoted} ${transform.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala
new file mode 100644
index 000000000000..8df88765a986
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+case class DropTagExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ tag: String,
+ ifExists: Boolean) extends LeafV2CommandExec {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val ref = iceberg.table().refs().get(tag)
+ if (ref != null || !ifExists) {
+ iceberg.table().manageSnapshots().removeTag(tag).commit()
+ }
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot drop tag on non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropTag tag: ${tag} for table: ${ident.quoted}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala
new file mode 100644
index 000000000000..c35af1486fc7
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.NoSuchViewException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+
+
+case class DropV2ViewExec(
+ catalog: ViewCatalog,
+ ident: Identifier,
+ ifExists: Boolean) extends LeafV2CommandExec {
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ val dropped = catalog.dropView(ident)
+ if (!dropped && !ifExists) {
+ throw new NoSuchViewException(ident)
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"DropV2View: ${ident}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
new file mode 100644
index 000000000000..6ee3d0f645bc
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.Spark3Util
+import org.apache.iceberg.spark.SparkCatalog
+import org.apache.iceberg.spark.SparkSessionCatalog
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.Strategy
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException
+import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier
+import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField
+import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch
+import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag
+import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation
+import org.apache.spark.sql.catalyst.plans.logical.DropBranch
+import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields
+import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField
+import org.apache.spark.sql.catalyst.plans.logical.DropTag
+import org.apache.spark.sql.catalyst.plans.logical.IcebergCall
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalesce
+import org.apache.spark.sql.catalyst.plans.logical.RenameTable
+import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField
+import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields
+import org.apache.spark.sql.catalyst.plans.logical.SetViewProperties
+import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering
+import org.apache.spark.sql.catalyst.plans.logical.ShowCreateTable
+import org.apache.spark.sql.catalyst.plans.logical.ShowTableProperties
+import org.apache.spark.sql.catalyst.plans.logical.UnsetViewProperties
+import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView
+import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View
+import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.execution.OrderAwareCoalesceExec
+import org.apache.spark.sql.execution.SparkPlan
+import scala.jdk.CollectionConverters._
+
+case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy with PredicateHelper {
+
+ override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+ case c @ IcebergCall(procedure, args) =>
+ val input = buildInternalRow(args)
+ CallExec(c.output, procedure, input) :: Nil
+
+ case AddPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform, name) =>
+ AddPartitionFieldExec(catalog, ident, transform, name) :: Nil
+
+ case CreateOrReplaceBranch(
+ IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) =>
+ CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil
+
+ case CreateOrReplaceTag(
+ IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) =>
+ CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil
+
+ case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) =>
+ DropBranchExec(catalog, ident, branch, ifExists) :: Nil
+
+ case DropTag(IcebergCatalogAndIdentifier(catalog, ident), tag, ifExists) =>
+ DropTagExec(catalog, ident, tag, ifExists) :: Nil
+
+ case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) =>
+ DropPartitionFieldExec(catalog, ident, transform) :: Nil
+
+ case ReplacePartitionField(IcebergCatalogAndIdentifier(catalog, ident), transformFrom, transformTo, name) =>
+ ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil
+
+ case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) =>
+ SetIdentifierFieldsExec(catalog, ident, fields) :: Nil
+
+ case DropIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) =>
+ DropIdentifierFieldsExec(catalog, ident, fields) :: Nil
+
+ case SetWriteDistributionAndOrdering(
+ IcebergCatalogAndIdentifier(catalog, ident), distributionMode, ordering) =>
+ SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil
+
+ case OrderAwareCoalesce(numPartitions, coalescer, child) =>
+ OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil
+
+ case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView@true) =>
+ val newIdent = Spark3Util.catalogAndIdentifier(spark, newName.toList.asJava)
+ if (oldCatalog.name != newIdent.catalog().name()) {
+ throw new IcebergAnalysisException(
+ s"Cannot move view between catalogs: from=${oldCatalog.name} and to=${newIdent.catalog().name()}")
+ }
+ RenameV2ViewExec(oldCatalog, oldIdent, newIdent.identifier()) :: Nil
+
+ case DropIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), ifExists) =>
+ DropV2ViewExec(viewCatalog, ident, ifExists) :: Nil
+
+ case CreateIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), queryText, query,
+ columnAliases, columnComments, queryColumnNames, comment, properties, allowExisting, replace, _) =>
+ CreateV2ViewExec(
+ catalog = viewCatalog,
+ ident = ident,
+ queryText = queryText,
+ columnAliases = columnAliases,
+ columnComments = columnComments,
+ queryColumnNames = queryColumnNames,
+ viewSchema = query.schema,
+ comment = comment,
+ properties = properties,
+ allowExisting = allowExisting,
+ replace = replace) :: Nil
+
+ case DescribeRelation(ResolvedV2View(catalog, ident), _, isExtended, output) =>
+ DescribeV2ViewExec(output, catalog.loadView(ident), isExtended) :: Nil
+
+ case ShowTableProperties(ResolvedV2View(catalog, ident), propertyKey, output) =>
+ ShowV2ViewPropertiesExec(output, catalog.loadView(ident), propertyKey) :: Nil
+
+ case ShowIcebergViews(ResolvedNamespace(catalog: ViewCatalog, namespace, _), pattern, output) =>
+ ShowV2ViewsExec(output, catalog, namespace, pattern) :: Nil
+
+ case ShowCreateTable(ResolvedV2View(catalog, ident), _, output) =>
+ ShowCreateV2ViewExec(output, catalog.loadView(ident)) :: Nil
+
+ case SetViewProperties(ResolvedV2View(catalog, ident), properties) =>
+ AlterV2ViewSetPropertiesExec(catalog, ident, properties) :: Nil
+
+ case UnsetViewProperties(ResolvedV2View(catalog, ident), propertyKeys, ifExists) =>
+ AlterV2ViewUnsetPropertiesExec(catalog, ident, propertyKeys, ifExists) :: Nil
+
+ case _ => Nil
+ }
+
+ private def buildInternalRow(exprs: Seq[Expression]): InternalRow = {
+ val values = new Array[Any](exprs.size)
+ for (index <- exprs.indices) {
+ values(index) = exprs(index).eval()
+ }
+ new GenericInternalRow(values)
+ }
+
+ private object IcebergCatalogAndIdentifier {
+ def unapply(identifier: Seq[String]): Option[(TableCatalog, Identifier)] = {
+ val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(spark, identifier.asJava)
+ catalogAndIdentifier.catalog match {
+ case icebergCatalog: SparkCatalog =>
+ Some((icebergCatalog, catalogAndIdentifier.identifier))
+ case icebergCatalog: SparkSessionCatalog[_] =>
+ Some((icebergCatalog, catalogAndIdentifier.identifier))
+ case _ =>
+ None
+ }
+ }
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala
new file mode 100644
index 000000000000..61d362044c3c
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+
+
+case class RenameV2ViewExec(
+ catalog: ViewCatalog,
+ oldIdent: Identifier,
+ newIdent: Identifier) extends LeafV2CommandExec {
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.renameView(oldIdent, newIdent)
+
+ Seq.empty
+ }
+
+
+ override def simpleString(maxFields: Int): String = {
+ s"RenameV2View ${oldIdent} to {newIdent}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala
new file mode 100644
index 000000000000..fcae0a5defc4
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.Spark3Util
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+import org.apache.spark.sql.connector.expressions.FieldReference
+import org.apache.spark.sql.connector.expressions.IdentityTransform
+import org.apache.spark.sql.connector.expressions.Transform
+
+case class ReplacePartitionFieldExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ transformFrom: Transform,
+ transformTo: Transform,
+ name: Option[String]) extends LeafV2CommandExec {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val schema = iceberg.table.schema
+ transformFrom match {
+ case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null =>
+ // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name
+ iceberg.table.updateSpec()
+ .removeField(parts.head)
+ .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo))
+ .commit()
+
+ case _ =>
+ iceberg.table.updateSpec()
+ .removeField(Spark3Util.toIcebergTerm(transformFrom))
+ .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo))
+ .commit()
+ }
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot replace partition field in non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"ReplacePartitionField ${catalog.name}.${ident.quoted} ${transformFrom.describe} " +
+ s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala
new file mode 100644
index 000000000000..b50550ad38ef
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+import scala.jdk.CollectionConverters._
+
+case class SetIdentifierFieldsExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ fields: Seq[String]) extends LeafV2CommandExec {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ iceberg.table.updateSchema()
+ .setIdentifierFields(fields.asJava)
+ .commit();
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot set identifier fields in non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"SetIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})";
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala
new file mode 100644
index 000000000000..c9004ddc5bda
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.iceberg.DistributionMode
+import org.apache.iceberg.NullOrder
+import org.apache.iceberg.SortDirection
+import org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE
+import org.apache.iceberg.expressions.Term
+import org.apache.iceberg.spark.SparkUtil
+import org.apache.iceberg.spark.source.SparkTable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+case class SetWriteDistributionAndOrderingExec(
+ catalog: TableCatalog,
+ ident: Identifier,
+ distributionMode: Option[DistributionMode],
+ sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec {
+
+ import CatalogV2Implicits._
+
+ override lazy val output: Seq[Attribute] = Nil
+
+ override protected def run(): Seq[InternalRow] = {
+ catalog.loadTable(ident) match {
+ case iceberg: SparkTable =>
+ val txn = iceberg.table.newTransaction()
+
+ val orderBuilder = txn.replaceSortOrder().caseSensitive(SparkUtil.caseSensitive(session))
+ sortOrder.foreach {
+ case (term, SortDirection.ASC, nullOrder) =>
+ orderBuilder.asc(term, nullOrder)
+ case (term, SortDirection.DESC, nullOrder) =>
+ orderBuilder.desc(term, nullOrder)
+ }
+ orderBuilder.commit()
+
+ distributionMode.foreach { mode =>
+ txn.updateProperties()
+ .set(WRITE_DISTRIBUTION_MODE, mode.modeName())
+ .commit()
+ }
+
+ txn.commitTransaction()
+
+ case table =>
+ throw new UnsupportedOperationException(s"Cannot set write order of non-Iceberg table: $table")
+ }
+
+ Nil
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ val tableIdent = s"${catalog.name}.${ident.quoted}"
+ val order = sortOrder.map {
+ case (term, direction, nullOrder) => s"$term $direction $nullOrder"
+ }.mkString(", ")
+ s"SetWriteDistributionAndOrdering $tableIdent $distributionMode $order"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala
new file mode 100644
index 000000000000..3be0f150313b
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString
+import org.apache.spark.sql.connector.catalog.View
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.execution.LeafExecNode
+import scala.collection.JavaConverters._
+
+case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View)
+ extends V2CommandExec with LeafExecNode {
+
+ override protected def run(): Seq[InternalRow] = {
+ val builder = new StringBuilder
+ builder ++= s"CREATE VIEW ${view.name} "
+ showColumns(view, builder)
+ showComment(view, builder)
+ showProperties(view, builder)
+ builder ++= s"AS\n${view.query}\n"
+
+ Seq(toCatalystRow(builder.toString))
+ }
+
+ private def showColumns(view: View, builder: StringBuilder): Unit = {
+ val columns = concatByMultiLines(
+ view.schema().fields
+ .map(x => s"${x.name}${x.getComment().map(c => s" COMMENT '$c'").getOrElse("")}"))
+ builder ++= columns
+ }
+
+ private def showComment(view: View, builder: StringBuilder): Unit = {
+ Option(view.properties.get(ViewCatalog.PROP_COMMENT))
+ .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n")
+ .foreach(builder.append)
+ }
+
+ private def showProperties(
+ view: View,
+ builder: StringBuilder): Unit = {
+ val showProps = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala
+ if (showProps.nonEmpty) {
+ val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map {
+ case (key, value) =>
+ s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'"
+ }
+
+ builder ++= "TBLPROPERTIES "
+ builder ++= concatByMultiLines(props)
+ }
+ }
+
+ private def concatByMultiLines(iter: Iterable[String]): String = {
+ iter.mkString("(\n ", ",\n ", ")\n")
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"ShowCreateV2ViewExec"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala
new file mode 100644
index 000000000000..89fafe99efc8
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.View
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.execution.LeafExecNode
+import scala.collection.JavaConverters._
+
+case class ShowV2ViewPropertiesExec(
+ output: Seq[Attribute],
+ view: View,
+ propertyKey: Option[String]) extends V2CommandExec with LeafExecNode {
+
+ override protected def run(): Seq[InternalRow] = {
+ propertyKey match {
+ case Some(p) =>
+ val propValue = properties.getOrElse(p,
+ s"View ${view.name()} does not have property: $p")
+ Seq(toCatalystRow(p, propValue))
+ case None =>
+ properties.map {
+ case (k, v) => toCatalystRow(k, v)
+ }.toSeq
+ }
+ }
+
+
+ private def properties = {
+ view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"ShowV2ViewPropertiesExec"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala
new file mode 100644
index 000000000000..a0699df13090
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.StringUtils
+import org.apache.spark.sql.connector.catalog.ViewCatalog
+import org.apache.spark.sql.execution.LeafExecNode
+import org.apache.spark.sql.internal.SQLConf
+import scala.collection.mutable.ArrayBuffer
+
+case class ShowV2ViewsExec(
+ output: Seq[Attribute],
+ catalog: ViewCatalog,
+ namespace: Seq[String],
+ pattern: Option[String]) extends V2CommandExec with LeafExecNode {
+
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+ override protected def run(): Seq[InternalRow] = {
+ val rows = new ArrayBuffer[InternalRow]()
+
+ // handle GLOBAL VIEWS
+ val globalTemp: String = SQLConf.get.globalTempDatabase
+ if (namespace.nonEmpty && globalTemp == namespace.head) {
+ pattern.map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p))
+ .getOrElse(session.sessionState.catalog.globalTempViewManager.listViewNames("*"))
+ .map(name => rows += toCatalystRow(globalTemp, name, true))
+ } else {
+ val views = catalog.listViews(namespace: _*)
+ views.map { view =>
+ if (pattern.map(StringUtils.filterPattern(Seq(view.name()), _).nonEmpty).getOrElse(true)) {
+ rows += toCatalystRow(view.namespace().quoted, view.name(), false)
+ }
+ }
+ }
+
+ // include TEMP VIEWS
+ pattern.map(p => session.sessionState.catalog.listLocalTempViews(p))
+ .getOrElse(session.sessionState.catalog.listLocalTempViews("*"))
+ .map(v => rows += toCatalystRow(v.database.toArray.quoted, v.table, true))
+
+ rows.toSeq
+ }
+
+ override def simpleString(maxFields: Int): String = {
+ s"ShowV2ViewsExec"
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java
new file mode 100644
index 000000000000..8918dfec6584
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Objects;
+
+public class Employee {
+ private Integer id;
+ private String dep;
+
+ public Employee() {}
+
+ public Employee(Integer id, String dep) {
+ this.id = id;
+ this.dep = dep;
+ }
+
+ public Integer getId() {
+ return id;
+ }
+
+ public void setId(Integer id) {
+ this.id = id;
+ }
+
+ public String getDep() {
+ return dep;
+ }
+
+ public void setDep(String dep) {
+ this.dep = dep;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ } else if (other == null || getClass() != other.getClass()) {
+ return false;
+ }
+
+ Employee employee = (Employee) other;
+ return Objects.equals(id, employee.id) && Objects.equals(dep, employee.dep);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(id, dep);
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java
new file mode 100644
index 000000000000..578845e3da2b
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java
@@ -0,0 +1,70 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.CatalogTestBase;
+import org.apache.iceberg.spark.TestBase;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.jupiter.api.BeforeAll;
+
+public abstract class ExtensionsTestBase extends CatalogTestBase {
+
+ private static final Random RANDOM = ThreadLocalRandom.current();
+
+ @BeforeAll
+ public static void startMetastoreAndSpark() {
+ TestBase.metastore = new TestHiveMetastore();
+ metastore.start();
+ TestBase.hiveConf = metastore.hiveConf();
+
+ TestBase.spark.close();
+
+ TestBase.spark =
+ SparkSession.builder()
+ .master("local[2]")
+ .config("spark.testing", "true")
+ .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+ .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName())
+ .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
+ .config("spark.sql.shuffle.partitions", "4")
+ .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+ .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+ .config(
+ SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))
+ .enableHiveSupport()
+ .getOrCreate();
+
+ TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+ TestBase.catalog =
+ (HiveCatalog)
+ CatalogUtil.loadCatalog(
+ HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java
new file mode 100644
index 000000000000..de4acd74a7ed
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.extensions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.UUID;
+import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile;
+import org.apache.iceberg.PartitionStatisticsFile;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class ProcedureUtil {
+
+ private ProcedureUtil() {}
+
+ static PartitionStatisticsFile writePartitionStatsFile(
+ long snapshotId, String statsLocation, FileIO fileIO) {
+ PositionOutputStream positionOutputStream;
+ try {
+ positionOutputStream = fileIO.newOutputFile(statsLocation).create();
+ positionOutputStream.close();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
+ return ImmutableGenericPartitionStatisticsFile.builder()
+ .snapshotId(snapshotId)
+ .fileSizeInBytes(42L)
+ .path(statsLocation)
+ .build();
+ }
+
+ static String statsFileLocation(String tableLocation) {
+ String statsFileName = "stats-file-" + UUID.randomUUID();
+ return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName;
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java
new file mode 100644
index 000000000000..830d07d86eab
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java
@@ -0,0 +1,107 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import static scala.collection.JavaConverters.seqAsJavaListConverter;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.spark.sql.catalyst.expressions.Expression;
+import org.apache.spark.sql.execution.CommandResultExec;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper;
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec;
+import scala.PartialFunction;
+import scala.collection.Seq;
+
+public class SparkPlanUtil {
+
+ private static final AdaptiveSparkPlanHelper SPARK_HELPER = new AdaptiveSparkPlanHelper() {};
+
+ private SparkPlanUtil() {}
+
+ public static List collectLeaves(SparkPlan plan) {
+ return toJavaList(SPARK_HELPER.collectLeaves(actualPlan(plan)));
+ }
+
+ public static List collectBatchScans(SparkPlan plan) {
+ List leaves = collectLeaves(plan);
+ return leaves.stream()
+ .filter(scan -> scan instanceof BatchScanExec)
+ .collect(Collectors.toList());
+ }
+
+ private static SparkPlan actualPlan(SparkPlan plan) {
+ if (plan instanceof CommandResultExec) {
+ return ((CommandResultExec) plan).commandPhysicalPlan();
+ } else {
+ return plan;
+ }
+ }
+
+ public static List collectExprs(
+ SparkPlan sparkPlan, Predicate predicate) {
+ Seq> seq =
+ SPARK_HELPER.collect(
+ sparkPlan,
+ new PartialFunction>() {
+ @Override
+ public List apply(SparkPlan plan) {
+ List exprs = Lists.newArrayList();
+
+ for (Expression expr : toJavaList(plan.expressions())) {
+ exprs.addAll(collectExprs(expr, predicate));
+ }
+
+ return exprs;
+ }
+
+ @Override
+ public boolean isDefinedAt(SparkPlan plan) {
+ return true;
+ }
+ });
+ return toJavaList(seq).stream().flatMap(Collection::stream).collect(Collectors.toList());
+ }
+
+ private static List collectExprs(
+ Expression expression, Predicate predicate) {
+ Seq seq =
+ expression.collect(
+ new PartialFunction() {
+ @Override
+ public Expression apply(Expression expr) {
+ return expr;
+ }
+
+ @Override
+ public boolean isDefinedAt(Expression expr) {
+ return predicate.test(expr);
+ }
+ });
+ return toJavaList(seq);
+ }
+
+ private static List toJavaList(Seq seq) {
+ return seqAsJavaListConverter(seq).asJava();
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
new file mode 100644
index 000000000000..7af9dfc58737
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
@@ -0,0 +1,400 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import static org.apache.iceberg.DataOperations.DELETE;
+import static org.apache.iceberg.DataOperations.OVERWRITE;
+import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
+import static org.apache.iceberg.PlanningMode.LOCAL;
+import static org.apache.iceberg.SnapshotSummary.ADDED_DELETE_FILES_PROP;
+import static org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP;
+import static org.apache.iceberg.SnapshotSummary.CHANGED_PARTITION_COUNT_PROP;
+import static org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP;
+import static org.apache.iceberg.TableProperties.DATA_PLANNING_MODE;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DELETE_PLANNING_MODE;
+import static org.apache.iceberg.TableProperties.ORC_VECTORIZATION_ENABLED;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Parameter;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.PlanningMode;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase {
+
+ private static final Random RANDOM = ThreadLocalRandom.current();
+
+ @Parameter(index = 3)
+ protected FileFormat fileFormat;
+
+ @Parameter(index = 4)
+ protected boolean vectorized;
+
+ @Parameter(index = 5)
+ protected String distributionMode;
+
+ @Parameter(index = 6)
+ protected boolean fanoutEnabled;
+
+ @Parameter(index = 7)
+ protected String branch;
+
+ @Parameter(index = 8)
+ protected PlanningMode planningMode;
+
+ @Parameters(
+ name =
+ "catalogName = {0}, implementation = {1}, config = {2},"
+ + " format = {3}, vectorized = {4}, distributionMode = {5},"
+ + " fanout = {6}, branch = {7}, planningMode = {8}")
+ public static Object[][] parameters() {
+ return new Object[][] {
+ {
+ "testhive",
+ SparkCatalog.class.getName(),
+ ImmutableMap.of(
+ "type", "hive",
+ "default-namespace", "default"),
+ FileFormat.ORC,
+ true,
+ WRITE_DISTRIBUTION_MODE_NONE,
+ true,
+ SnapshotRef.MAIN_BRANCH,
+ LOCAL
+ },
+ {
+ "testhive",
+ SparkCatalog.class.getName(),
+ ImmutableMap.of(
+ "type", "hive",
+ "default-namespace", "default"),
+ FileFormat.PARQUET,
+ true,
+ WRITE_DISTRIBUTION_MODE_NONE,
+ false,
+ "test",
+ DISTRIBUTED
+ },
+ {
+ "testhadoop",
+ SparkCatalog.class.getName(),
+ ImmutableMap.of("type", "hadoop"),
+ FileFormat.PARQUET,
+ RANDOM.nextBoolean(),
+ WRITE_DISTRIBUTION_MODE_HASH,
+ true,
+ null,
+ LOCAL
+ },
+ {
+ "spark_catalog",
+ SparkSessionCatalog.class.getName(),
+ ImmutableMap.of(
+ "type", "hive",
+ "default-namespace", "default",
+ "clients", "1",
+ "parquet-enabled", "false",
+ "cache-enabled",
+ "false" // Spark will delete tables using v1, leaving the cache out of sync
+ ),
+ FileFormat.AVRO,
+ false,
+ WRITE_DISTRIBUTION_MODE_RANGE,
+ false,
+ "test",
+ DISTRIBUTED
+ }
+ };
+ }
+
+ protected abstract Map extraTableProperties();
+
+ protected void initTable() {
+ sql(
+ "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s')",
+ tableName,
+ DEFAULT_FILE_FORMAT,
+ fileFormat,
+ WRITE_DISTRIBUTION_MODE,
+ distributionMode,
+ SPARK_WRITE_PARTITIONED_FANOUT_ENABLED,
+ String.valueOf(fanoutEnabled),
+ DATA_PLANNING_MODE,
+ planningMode.modeName(),
+ DELETE_PLANNING_MODE,
+ planningMode.modeName());
+
+ switch (fileFormat) {
+ case PARQUET:
+ sql(
+ "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')",
+ tableName, PARQUET_VECTORIZATION_ENABLED, vectorized);
+ break;
+ case ORC:
+ sql(
+ "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')",
+ tableName, ORC_VECTORIZATION_ENABLED, vectorized);
+ break;
+ case AVRO:
+ assertThat(vectorized).isFalse();
+ break;
+ }
+
+ Map props = extraTableProperties();
+ props.forEach(
+ (prop, value) -> {
+ sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", tableName, prop, value);
+ });
+ }
+
+ protected void createAndInitTable(String schema) {
+ createAndInitTable(schema, null);
+ }
+
+ protected void createAndInitTable(String schema, String jsonData) {
+ createAndInitTable(schema, "", jsonData);
+ }
+
+ protected void createAndInitTable(String schema, String partitioning, String jsonData) {
+ sql("CREATE TABLE %s (%s) USING iceberg %s", tableName, schema, partitioning);
+ initTable();
+
+ if (jsonData != null) {
+ try {
+ Dataset ds = toDS(schema, jsonData);
+ ds.coalesce(1).writeTo(tableName).append();
+ createBranchIfNeeded();
+ } catch (NoSuchTableException e) {
+ throw new RuntimeException("Failed to write data", e);
+ }
+ }
+ }
+
+ protected void append(String table, String jsonData) {
+ append(table, null, jsonData);
+ }
+
+ protected void append(String table, String schema, String jsonData) {
+ try {
+ Dataset ds = toDS(schema, jsonData);
+ ds.coalesce(1).writeTo(table).append();
+ } catch (NoSuchTableException e) {
+ throw new RuntimeException("Failed to write data", e);
+ }
+ }
+
+ protected void createOrReplaceView(String name, String jsonData) {
+ createOrReplaceView(name, null, jsonData);
+ }
+
+ protected void createOrReplaceView(String name, String schema, String jsonData) {
+ Dataset ds = toDS(schema, jsonData);
+ ds.createOrReplaceTempView(name);
+ }
+
+ protected void createOrReplaceView(String name, List data, Encoder encoder) {
+ spark.createDataset(data, encoder).createOrReplaceTempView(name);
+ }
+
+ private Dataset toDS(String schema, String jsonData) {
+ List jsonRows =
+ Arrays.stream(jsonData.split("\n"))
+ .filter(str -> !str.trim().isEmpty())
+ .collect(Collectors.toList());
+ Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING());
+
+ if (schema != null) {
+ return spark.read().schema(schema).json(jsonDS);
+ } else {
+ return spark.read().json(jsonDS);
+ }
+ }
+
+ protected void validateDelete(
+ Snapshot snapshot, String changedPartitionCount, String deletedDataFiles) {
+ validateSnapshot(snapshot, DELETE, changedPartitionCount, deletedDataFiles, null, null);
+ }
+
+ protected void validateCopyOnWrite(
+ Snapshot snapshot,
+ String changedPartitionCount,
+ String deletedDataFiles,
+ String addedDataFiles) {
+ String operation = null == addedDataFiles && null != deletedDataFiles ? DELETE : OVERWRITE;
+ validateSnapshot(
+ snapshot, operation, changedPartitionCount, deletedDataFiles, null, addedDataFiles);
+ }
+
+ protected void validateMergeOnRead(
+ Snapshot snapshot,
+ String changedPartitionCount,
+ String addedDeleteFiles,
+ String addedDataFiles) {
+ String operation = null == addedDataFiles && null != addedDeleteFiles ? DELETE : OVERWRITE;
+ validateSnapshot(
+ snapshot, operation, changedPartitionCount, null, addedDeleteFiles, addedDataFiles);
+ }
+
+ protected void validateSnapshot(
+ Snapshot snapshot,
+ String operation,
+ String changedPartitionCount,
+ String deletedDataFiles,
+ String addedDeleteFiles,
+ String addedDataFiles) {
+ assertThat(snapshot.operation()).as("Operation must match").isEqualTo(operation);
+ validateProperty(snapshot, CHANGED_PARTITION_COUNT_PROP, changedPartitionCount);
+ validateProperty(snapshot, DELETED_FILES_PROP, deletedDataFiles);
+ validateProperty(snapshot, ADDED_DELETE_FILES_PROP, addedDeleteFiles);
+ validateProperty(snapshot, ADDED_FILES_PROP, addedDataFiles);
+ }
+
+ protected void validateProperty(Snapshot snapshot, String property, Set expectedValues) {
+ String actual = snapshot.summary().get(property);
+ assertThat(actual)
+ .as(
+ "Snapshot property "
+ + property
+ + " has unexpected value, actual = "
+ + actual
+ + ", expected one of : "
+ + String.join(",", expectedValues))
+ .isIn(expectedValues);
+ }
+
+ protected void validateProperty(Snapshot snapshot, String property, String expectedValue) {
+ if (null == expectedValue) {
+ assertThat(snapshot.summary()).doesNotContainKey(property);
+ } else {
+ assertThat(snapshot.summary())
+ .as("Snapshot property " + property + " has unexpected value.")
+ .containsEntry(property, expectedValue);
+ }
+ }
+
+ protected void sleep(long millis) {
+ try {
+ Thread.sleep(millis);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected DataFile writeDataFile(Table table, List records) {
+ try {
+ OutputFile file =
+ Files.localOutput(
+ temp.resolve(fileFormat.addExtension(UUID.randomUUID().toString())).toFile());
+
+ DataWriter dataWriter =
+ Parquet.writeData(file)
+ .forTable(table)
+ .createWriterFunc(GenericParquetWriter::buildWriter)
+ .overwrite()
+ .build();
+
+ try {
+ for (GenericRecord record : records) {
+ dataWriter.write(record);
+ }
+ } finally {
+ dataWriter.close();
+ }
+
+ return dataWriter.toDataFile();
+
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ @Override
+ protected String commitTarget() {
+ return branch == null ? tableName : String.format("%s.branch_%s", tableName, branch);
+ }
+
+ @Override
+ protected String selectTarget() {
+ return branch == null ? tableName : String.format("%s VERSION AS OF '%s'", tableName, branch);
+ }
+
+ protected void createBranchIfNeeded() {
+ if (branch != null && !branch.equals(SnapshotRef.MAIN_BRANCH)) {
+ sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branch);
+ }
+ }
+
+ // ORC currently does not support vectorized reads with deletes
+ protected boolean supportsVectorization() {
+ return vectorized && (isParquet() || isCopyOnWrite());
+ }
+
+ private boolean isParquet() {
+ return fileFormat.equals(FileFormat.PARQUET);
+ }
+
+ private boolean isCopyOnWrite() {
+ return extraTableProperties().containsValue(RowLevelOperationMode.COPY_ON_WRITE.modeName());
+ }
+
+ protected void assertAllBatchScansVectorized(SparkPlan plan) {
+ List batchScans = SparkPlanUtil.collectBatchScans(plan);
+ assertThat(batchScans).hasSizeGreaterThan(0).allMatch(SparkPlan::supportsColumnar);
+ }
+}
diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
new file mode 100644
index 000000000000..920c2f55eaaf
--- /dev/null
+++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
@@ -0,0 +1,1148 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Parameter;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.joda.time.DateTime;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.io.TempDir;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestAddFilesProcedure extends ExtensionsTestBase {
+
+ @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}")
+ public static Object[][] parameters() {
+ return new Object[][] {
+ {
+ SparkCatalogConfig.HIVE.catalogName(),
+ SparkCatalogConfig.HIVE.implementation(),
+ SparkCatalogConfig.HIVE.properties(),
+ 1
+ },
+ {
+ SparkCatalogConfig.HADOOP.catalogName(),
+ SparkCatalogConfig.HADOOP.implementation(),
+ SparkCatalogConfig.HADOOP.properties(),
+ 2
+ },
+ {
+ SparkCatalogConfig.SPARK.catalogName(),
+ SparkCatalogConfig.SPARK.implementation(),
+ SparkCatalogConfig.SPARK.properties(),
+ 2
+ }
+ };
+ }
+
+ @Parameter(index = 3)
+ private int formatVersion;
+
+ private final String sourceTableName = "source_table";
+ private File fileTableDir;
+
+ @TempDir private Path temp;
+
+ @BeforeEach
+ public void setupTempDirs() {
+ fileTableDir = temp.toFile();
+ }
+
+ @AfterEach
+ public void dropTables() {
+ sql("DROP TABLE IF EXISTS %s PURGE", sourceTableName);
+ sql("DROP TABLE IF EXISTS %s", tableName);
+ }
+
+ @TestTemplate
+ public void addDataUnpartitioned() {
+ createUnpartitionedFileTable("parquet");
+
+ createIcebergTable("id Integer, name String, dept String, subdept String");
+
+ List