diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml
index 1b756ff419a0..e34b73101d92 100644
--- a/.github/workflows/spark-ci.yml
+++ b/.github/workflows/spark-ci.yml
@@ -60,7 +60,7 @@ jobs:
strategy:
matrix:
jvm: [8, 11]
- spark: ['3.1', '3.2', '3.3', '3.4', '3.5']
+ spark: ['3.2', '3.3', '3.4', '3.5']
env:
SPARK_LOCAL_IP: localhost
steps:
diff --git a/.gitignore b/.gitignore
index 74d4d1c3e438..59977da38ef5 100644
--- a/.gitignore
+++ b/.gitignore
@@ -28,7 +28,6 @@ lib/
site/site
# benchmark output
-spark/v3.1/spark/benchmark/*
spark/v3.2/spark/benchmark/*
spark/v3.3/spark/benchmark/*
spark/v3.3/spark-extensions/benchmark/*
diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh
index 0ec940f64fae..4b4767784b41 100755
--- a/dev/stage-binaries.sh
+++ b/dev/stage-binaries.sh
@@ -20,7 +20,7 @@
SCALA_VERSION=2.12
FLINK_VERSIONS=1.15,1.16,1.17
-SPARK_VERSIONS=3.1,3.2,3.3,3.4
+SPARK_VERSIONS=3.2,3.3,3.4
HIVE_VERSIONS=2,3
./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DhiveVersions=$HIVE_VERSIONS publishApachePublicationToMavenRepository
diff --git a/gradle.properties b/gradle.properties
index 1733f2d26a40..1bce26270354 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -21,7 +21,7 @@ systemProp.knownFlinkVersions=1.15,1.16,1.17
systemProp.defaultHiveVersions=2
systemProp.knownHiveVersions=2,3
systemProp.defaultSparkVersions=3.5
-systemProp.knownSparkVersions=3.1,3.2,3.3,3.4,3.5
+systemProp.knownSparkVersions=3.2,3.3,3.4,3.5
systemProp.defaultScalaVersion=2.12
systemProp.knownScalaVersions=2.12,2.13
org.gradle.parallel=true
diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml
index 2deb6c3c2d6f..a0fe0df3ba7d 100644
--- a/gradle/libs.versions.toml
+++ b/gradle/libs.versions.toml
@@ -59,7 +59,6 @@ s3mock-junit5 = "2.11.0"
scala-collection-compat = "2.11.0"
slf4j = "1.7.36"
snowflake-jdbc = "3.13.30"
-spark-hive31 = "3.1.3"
spark-hive32 = "3.2.2"
spark-hive33 = "3.3.2"
spark-hive34 = "3.4.1"
diff --git a/jmh.gradle b/jmh.gradle
index a7e64610a23f..aa60b93b331d 100644
--- a/jmh.gradle
+++ b/jmh.gradle
@@ -25,10 +25,6 @@ def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getPro
def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion")
def jmhProjects = [project(":iceberg-core")]
-if (sparkVersions.contains("3.1")) {
- jmhProjects.add(project(":iceberg-spark:iceberg-spark-3.1_2.12"))
-}
-
if (sparkVersions.contains("3.2")) {
jmhProjects.add(project(":iceberg-spark:iceberg-spark-3.2_${scalaVersion}"))
}
diff --git a/settings.gradle b/settings.gradle
index 8b3b8b79d7c3..9ab130413077 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -133,18 +133,6 @@ if (flinkVersions.contains("1.17")) {
project(":iceberg-flink:flink-runtime-1.17").name = "iceberg-flink-runtime-1.17"
}
-if (sparkVersions.contains("3.1")) {
- include ':iceberg-spark:spark-3.1_2.12'
- include ':iceberg-spark:spark-extensions-3.1_2.12'
- include ':iceberg-spark:spark-runtime-3.1_2.12'
- project(':iceberg-spark:spark-3.1_2.12').projectDir = file('spark/v3.1/spark')
- project(':iceberg-spark:spark-3.1_2.12').name = 'iceberg-spark-3.1_2.12'
- project(':iceberg-spark:spark-extensions-3.1_2.12').projectDir = file('spark/v3.1/spark-extensions')
- project(':iceberg-spark:spark-extensions-3.1_2.12').name = 'iceberg-spark-extensions-3.1_2.12'
- project(':iceberg-spark:spark-runtime-3.1_2.12').projectDir = file('spark/v3.1/spark-runtime')
- project(':iceberg-spark:spark-runtime-3.1_2.12').name = 'iceberg-spark-runtime-3.1_2.12'
-}
-
if (sparkVersions.contains("3.2")) {
include ":iceberg-spark:spark-3.2_${scalaVersion}"
include ":iceberg-spark:spark-extensions-3.2_${scalaVersion}"
diff --git a/spark/build.gradle b/spark/build.gradle
index 6077abf935b0..736c47e335ff 100644
--- a/spark/build.gradle
+++ b/spark/build.gradle
@@ -20,10 +20,6 @@
// add enabled Spark version modules to the build
def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getProperty("sparkVersions") : System.getProperty("defaultSparkVersions")).split(",")
-if (sparkVersions.contains("3.1")) {
- apply from: file("$projectDir/v3.1/build.gradle")
-}
-
if (sparkVersions.contains("3.2")) {
apply from: file("$projectDir/v3.2/build.gradle")
}
diff --git a/spark/v3.1/build.gradle b/spark/v3.1/build.gradle
deleted file mode 100644
index 8ab31995ef17..000000000000
--- a/spark/v3.1/build.gradle
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion")
-
-def sparkProjects = [
- project(':iceberg-spark:iceberg-spark-3.1_2.12'),
- project(":iceberg-spark:iceberg-spark-extensions-3.1_2.12"),
- project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12')
-]
-
-configure(sparkProjects) {
-
- configurations {
- all {
- resolutionStrategy {
- force "com.fasterxml.jackson.module:jackson-module-scala_${scalaVersion}:${libs.versions.jackson211.get()}"
- force "com.fasterxml.jackson.core:jackson-paranamer:${libs.versions.jackson211.get()}"
- force "com.fasterxml.jackson.core:jackson-databind:${libs.versions.jackson211.get()}"
- force "com.fasterxml.jackson.core:jackson-core:${libs.versions.jackson211.get()}"
- }
- }
- }
-}
-
-project(':iceberg-spark:iceberg-spark-3.1_2.12') {
- apply plugin: 'scala'
- apply plugin: 'com.github.alisiikh.scalastyle'
-
- sourceSets {
- main {
- scala.srcDirs = ['src/main/scala', 'src/main/java']
- java.srcDirs = []
- }
- }
-
- dependencies {
- implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
- api project(':iceberg-api')
- implementation project(':iceberg-common')
- implementation project(':iceberg-core')
- implementation project(':iceberg-data')
- implementation project(':iceberg-orc')
- implementation project(':iceberg-parquet')
- implementation(project(':iceberg-arrow')) {
- exclude group: 'io.netty', module: 'netty-buffer'
- exclude group: 'io.netty', module: 'netty-common'
- }
-
- compileOnly libs.errorprone.annotations
- compileOnly libs.avro.avro
- compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive31.get()}") {
- exclude group: 'org.apache.avro', module: 'avro'
- exclude group: 'org.apache.arrow'
- exclude group: 'org.apache.parquet'
- exclude group: 'io.netty', module: 'netty-buffer'
- exclude group: 'io.netty', module: 'netty-common'
- exclude group: 'org.roaringbitmap'
- }
-
- implementation libs.parquet.column
- implementation libs.parquet.hadoop
-
- implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") {
- exclude group: 'org.apache.hadoop'
- exclude group: 'commons-lang'
- // These artifacts are shaded and included in the orc-core fat jar
- exclude group: 'com.google.protobuf', module: 'protobuf-java'
- exclude group: 'org.apache.hive', module: 'hive-storage-api'
- }
-
- implementation(libs.arrow.vector) {
- exclude group: 'io.netty', module: 'netty-buffer'
- exclude group: 'io.netty', module: 'netty-common'
- exclude group: 'com.google.code.findbugs', module: 'jsr305'
- }
-
- // use netty-buffer compatible with Spark 3.1
- runtimeOnly libs.netty.buffer.compat
-
- testImplementation(libs.hadoop2.minicluster) {
- exclude group: 'org.apache.avro', module: 'avro'
- exclude group: 'io.netty', module: 'netty-buffer'
- exclude group: 'io.netty', module: 'netty-common'
- }
- testImplementation project(path: ':iceberg-hive-metastore')
- testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts')
- testImplementation libs.sqlite.jdbc
- }
-
- tasks.withType(Test) {
- // For vectorized reads
- // Allow unsafe memory access to avoid the costly check arrow does to check if index is within bounds
- systemProperty("arrow.enable_unsafe_memory_access", "true")
- // Disable expensive null check for every get(index) call.
- // Iceberg manages nullability checks itself instead of relying on arrow.
- systemProperty("arrow.enable_null_check_for_get", "false")
-
- // Vectorized reads need more memory
- maxHeapSize '2560m'
- }
-}
-
-project(":iceberg-spark:iceberg-spark-extensions-3.1_2.12") {
- apply plugin: 'java-library'
- apply plugin: 'scala'
- apply plugin: 'com.github.alisiikh.scalastyle'
- apply plugin: 'antlr'
-
- configurations {
- /*
- The Gradle Antlr plugin erroneously adds both antlr-build and runtime dependencies to the runtime path. This
- bug https://github.com/gradle/gradle/issues/820 exists because older versions of Antlr do not have separate
- runtime and implementation dependencies and they do not want to break backwards compatibility. So to only end up with
- the runtime dependency on the runtime classpath we remove the dependencies added by the plugin here. Then add
- the runtime dependency back to only the runtime configuration manually.
- */
- implementation {
- extendsFrom = extendsFrom.findAll { it != configurations.antlr }
- }
- }
-
- dependencies {
- compileOnly "org.scala-lang:scala-library"
- compileOnly project(path: ':iceberg-bundled-guava', configuration: 'shadow')
- compileOnly project(':iceberg-api')
- compileOnly project(':iceberg-core')
- compileOnly project(':iceberg-data')
- compileOnly project(':iceberg-orc')
- compileOnly project(':iceberg-common')
- compileOnly project(':iceberg-spark:iceberg-spark-3.1_2.12')
- compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive31.get()}") {
- exclude group: 'org.apache.avro', module: 'avro'
- exclude group: 'org.apache.arrow'
- exclude group: 'org.apache.parquet'
- exclude group: 'io.netty', module: 'netty-buffer'
- exclude group: 'io.netty', module: 'netty-common'
- exclude group: 'org.roaringbitmap'
- }
-
- testImplementation project(path: ':iceberg-hive-metastore')
- testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
-
- testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-orc', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
- testImplementation project(path: ':iceberg-spark:iceberg-spark-3.1_2.12', configuration: 'testArtifacts')
-
- testImplementation libs.avro.avro
-
- // Required because we remove antlr plugin dependencies from the compile configuration, see note above
- runtimeOnly libs.antlr.runtime
- antlr libs.antlr.antlr4
- }
-
- generateGrammarSource {
- maxHeapSize = "64m"
- arguments += ['-visitor', '-package', 'org.apache.spark.sql.catalyst.parser.extensions']
- }
-}
-
-project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') {
- apply plugin: 'com.github.johnrengelman.shadow'
-
- tasks.jar.dependsOn tasks.shadowJar
-
- sourceSets {
- integration {
- java.srcDir "$projectDir/src/integration/java"
- resources.srcDir "$projectDir/src/integration/resources"
- }
- }
-
- configurations {
- implementation {
- exclude group: 'org.apache.spark'
- // included in Spark
- exclude group: 'org.slf4j'
- exclude group: 'org.apache.commons'
- exclude group: 'commons-pool'
- exclude group: 'commons-codec'
- exclude group: 'org.xerial.snappy'
- exclude group: 'javax.xml.bind'
- exclude group: 'javax.annotation'
- exclude group: 'com.github.luben'
- exclude group: 'com.ibm.icu'
- exclude group: 'org.glassfish'
- exclude group: 'org.abego.treelayout'
- exclude group: 'org.antlr'
- }
- }
-
- dependencies {
- api project(':iceberg-api')
- implementation project(':iceberg-spark:iceberg-spark-3.1_2.12')
- implementation project(':iceberg-spark:iceberg-spark-extensions-3.1_2.12')
- implementation project(':iceberg-aws')
- implementation project(':iceberg-azure')
- implementation(project(':iceberg-aliyun')) {
- exclude group: 'edu.umd.cs.findbugs', module: 'findbugs'
- exclude group: 'org.apache.httpcomponents', module: 'httpclient'
- exclude group: 'commons-logging', module: 'commons-logging'
- }
- implementation project(':iceberg-gcp')
- implementation project(':iceberg-hive-metastore')
- implementation(project(':iceberg-nessie')) {
- exclude group: 'com.google.code.findbugs', module: 'jsr305'
- }
- implementation (project(':iceberg-snowflake')) {
- exclude group: 'net.snowflake' , module: 'snowflake-jdbc'
- }
-
- integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive31.get()}"
- integrationImplementation libs.junit.vintage.engine
- integrationImplementation libs.slf4j.simple
- integrationImplementation libs.assertj.core
- integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
- integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
- integrationImplementation project(path: ':iceberg-spark:iceberg-spark-3.1_2.12', configuration: 'testArtifacts')
- integrationImplementation project(path: ':iceberg-spark:iceberg-spark-extensions-3.1_2.12', configuration: 'testArtifacts')
- // Not allowed on our classpath, only the runtime jar is allowed
- integrationCompileOnly project(':iceberg-spark:iceberg-spark-extensions-3.1_2.12')
- integrationCompileOnly project(':iceberg-spark:iceberg-spark-3.1_2.12')
- integrationCompileOnly project(':iceberg-api')
- }
-
- shadowJar {
- configurations = [project.configurations.runtimeClasspath]
-
- zip64 true
-
- // include the LICENSE and NOTICE files for the shaded Jar
- from(projectDir) {
- include 'LICENSE'
- include 'NOTICE'
- }
-
- // Relocate dependencies to avoid conflicts
- relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone'
- relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers'
- relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
- relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
- relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
- relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'
- relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded'
- relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer'
- relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet'
- relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded'
- relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc'
- relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift'
- relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5'
- relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5'
- // relocate Arrow and related deps to shade Iceberg specific version
- relocate 'io.netty', 'org.apache.iceberg.shaded.io.netty'
- relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow'
- relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch'
- relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra'
- relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap'
-
- archiveClassifier.set(null)
- }
-
- task integrationTest(type: Test) {
- description = "Test Spark3 Runtime Jar against Spark 3.1"
- group = "verification"
- jvmArgs += project.property('extraJvmArgs')
- testClassesDirs = sourceSets.integration.output.classesDirs
- classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path)
- inputs.file(shadowJar.archiveFile.get().asFile.path)
- }
- integrationTest.dependsOn shadowJar
- check.dependsOn integrationTest
-
- jar {
- enabled = false
- }
-}
-
diff --git a/spark/v3.1/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 b/spark/v3.1/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4
deleted file mode 100644
index cf5d4588fc8d..000000000000
--- a/spark/v3.1/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4
+++ /dev/null
@@ -1,366 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- *
- * 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 DROP BRANCH (IF EXISTS)? identifier #dropBranch
- | ALTER TABLE multipartIdentifier createReplaceTagClause #createOrReplaceTag
- | ALTER TABLE multipartIdentifier DROP TAG (IF EXISTS)? identifier #dropTag
- ;
-
-createReplaceBranchClause
- : (CREATE OR)? REPLACE BRANCH identifier branchOptions
- | CREATE BRANCH (IF NOT EXISTS)? identifier branchOptions
- ;
-
-createReplaceTagClause
- : (CREATE OR)? REPLACE TAG identifier tagOptions
- | CREATE TAG (IF NOT EXISTS)? identifier tagOptions
- ;
-
-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
- ;
-
-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)* ')'
- ;
-
-booleanValue
- : TRUE | FALSE
- ;
-
-stringArray
- : ARRAY '(' constant (',' constant)* ')'
- ;
-
-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 | RETENTION | VERSION | WITH | IDENTIFIER_KW | FIELDS | SET | SNAPSHOT | SNAPSHOTS
- | TAG | TRUE | FALSE
- | MAP
- ;
-
-snapshotId
- : number
- ;
-
-timeUnit
- : DAYS
- | HOURS
- | MINUTES
- ;
-
-ADD: 'ADD';
-ALTER: 'ALTER';
-AS: 'AS';
-ASC: 'ASC';
-BRANCH: 'BRANCH';
-BY: 'BY';
-CALL: 'CALL';
-CREATE: 'CREATE';
-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';
-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/v3.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
deleted file mode 100644
index 30b5df5317db..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iceberg.spark.extensions
-
-import org.apache.spark.sql.SparkSessionExtensions
-import org.apache.spark.sql.catalyst.analysis.AlignRowLevelOperations
-import org.apache.spark.sql.catalyst.analysis.ProcedureArgumentCoercion
-import org.apache.spark.sql.catalyst.analysis.ResolveProcedures
-import org.apache.spark.sql.catalyst.analysis.RowLevelOperationsPredicateCheck
-import org.apache.spark.sql.catalyst.optimizer.OptimizeConditionsInRowLevelOperations
-import org.apache.spark.sql.catalyst.optimizer.PullupCorrelatedPredicatesInRowLevelOperations
-import org.apache.spark.sql.catalyst.optimizer.RewriteDelete
-import org.apache.spark.sql.catalyst.optimizer.RewriteMergeInto
-import org.apache.spark.sql.catalyst.optimizer.RewriteUpdate
-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 { _ => ProcedureArgumentCoercion }
- extensions.injectPostHocResolutionRule { spark => AlignRowLevelOperations }
- extensions.injectCheckRule { _ => RowLevelOperationsPredicateCheck }
-
- // optimizer extensions
- extensions.injectOptimizerRule { _ => OptimizeConditionsInRowLevelOperations }
- extensions.injectOptimizerRule { _ => PullupCorrelatedPredicatesInRowLevelOperations }
- extensions.injectOptimizerRule { spark => RewriteDelete(spark) }
- extensions.injectOptimizerRule { spark => RewriteUpdate(spark) }
- extensions.injectOptimizerRule { spark => RewriteMergeInto(spark) }
-
- // planner extensions
- extensions.injectPlannerStrategy { spark => ExtendedDataSourceV2Strategy(spark) }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala
deleted file mode 100644
index 6da3ba6323e2..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.plans.logical.Assignment
-import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
-import org.apache.spark.sql.catalyst.plans.logical.InsertAction
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
-import org.apache.spark.sql.catalyst.plans.logical.UpdateAction
-import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-import org.apache.spark.sql.internal.SQLConf
-
-case object AlignRowLevelOperations extends Rule[LogicalPlan]
- with AssignmentAlignmentSupport with CastSupport {
-
- override def conf: SQLConf = SQLConf.get
-
- override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
- case u: UpdateTable if u.resolved && isIcebergRelation(u.table)=>
- u.copy(assignments = alignAssignments(u.table, u.assignments))
-
- case m: MergeIntoTable if m.resolved && isIcebergRelation(m.targetTable) =>
- val alignedMatchedActions = m.matchedActions.map {
- case u @ UpdateAction(_, assignments) =>
- u.copy(assignments = alignAssignments(m.targetTable, assignments))
- case d: DeleteAction =>
- d
- case _ =>
- throw new AnalysisException("Matched actions can only contain UPDATE or DELETE")
- }
-
- val alignedNotMatchedActions = m.notMatchedActions.map {
- case i @ InsertAction(_, assignments) =>
- // check no nested columns are present
- val refs = assignments.map(_.key).map(asAssignmentReference)
- refs.foreach { ref =>
- if (ref.size > 1) {
- throw new AnalysisException(
- "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
- s"${ref.mkString("`", "`.`", "`")}")
- }
- }
-
- val colNames = refs.map(_.head)
-
- // check there are no duplicates
- val duplicateColNames = colNames.groupBy(identity).collect {
- case (name, matchingNames) if matchingNames.size > 1 => name
- }
-
- if (duplicateColNames.nonEmpty) {
- throw new AnalysisException(
- s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
- }
-
- // reorder assignments by the target table column order
- val assignmentMap = colNames.zip(assignments).toMap
- i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
-
- case _ =>
- throw new AnalysisException("Not matched actions can only contain INSERT")
- }
-
- m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
- }
-
- private def alignInsertActionAssignments(
- targetTable: LogicalPlan,
- assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
-
- val resolver = conf.resolver
-
- targetTable.output.map { targetAttr =>
- val assignment = assignmentMap
- .find { case (name, _) => resolver(name, targetAttr.name) }
- .map { case (_, assignment) => assignment }
-
- if (assignment.isEmpty) {
- throw new AnalysisException(
- s"Cannot find column '${targetAttr.name}' of the target table among " +
- s"the INSERT columns: ${assignmentMap.keys.mkString(", ")}. " +
- "INSERT clauses must provide values for all columns of the target table.")
- }
-
- val key = assignment.get.key
- val value = assignment.get.value
- Assignment(key, castIfNeeded(targetAttr, value, resolver))
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
deleted file mode 100644
index d61320ecfaf5..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.Alias
-import org.apache.spark.sql.catalyst.expressions.AnsiCast
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.catalyst.expressions.Cast
-import org.apache.spark.sql.catalyst.expressions.CreateNamedStruct
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.ExtractValue
-import org.apache.spark.sql.catalyst.expressions.GetStructField
-import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.expressions.NamedExpression
-import org.apache.spark.sql.catalyst.plans.logical.Assignment
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
-import org.apache.spark.sql.types.DataType
-import org.apache.spark.sql.types.StructField
-import org.apache.spark.sql.types.StructType
-import scala.collection.mutable
-
-trait AssignmentAlignmentSupport {
-
- def conf: SQLConf
-
- private case class ColumnUpdate(ref: Seq[String], expr: Expression)
-
- /**
- * Aligns assignments to match table columns.
- *
- * This method processes and reorders given assignments so that each target column gets
- * an expression it should be set to. If a column does not have a matching assignment,
- * it will be set to its current value. For example, if one passes a table with columns c1, c2
- * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
- *
- * This method also handles updates to nested columns. If there is an assignment to a particular
- * nested field, this method will construct a new struct with one field updated
- * preserving other fields that have not been modified. For example, if one passes a table with
- * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
- * this method will return c1 = c1, c2 = struct(c2.n1, 1).
- *
- * @param table a target table
- * @param assignments assignments to align
- * @return aligned assignments that match table columns
- */
- protected def alignAssignments(
- table: LogicalPlan,
- assignments: Seq[Assignment]): Seq[Assignment] = {
-
- val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
- val outputExprs = applyUpdates(table.output, columnUpdates)
- outputExprs.zip(table.output).map {
- case (expr, attr) => Assignment(attr, expr)
- }
- }
-
- private def applyUpdates(
- cols: Seq[NamedExpression],
- updates: Seq[ColumnUpdate],
- resolver: Resolver = conf.resolver,
- namePrefix: Seq[String] = Nil): Seq[Expression] = {
-
- // iterate through columns at the current level and find which column updates match
- cols.map { col =>
- // find matches for this column or any of its children
- val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
- prefixMatchedUpdates match {
- // if there is no exact match and no match for children, return the column as is
- case updates if updates.isEmpty =>
- col
-
- // if there is an exact match, return the assigned expression
- case Seq(update) if isExactMatch(update, col, resolver) =>
- castIfNeeded(col, update.expr, resolver)
-
- // if there are matches only for children
- case updates if !hasExactMatch(updates, col, resolver) =>
- col.dataType match {
- case StructType(fields) =>
- // build field expressions
- val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
- Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
- }
-
- // recursively apply this method on nested fields
- val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
- val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
-
- // construct a new struct with updated field expressions
- toNamedStruct(fields, updatedFieldExprs)
-
- case otherType =>
- val colName = (namePrefix :+ col.name).mkString(".")
- throw new AnalysisException(
- "Updating nested fields is only supported for StructType " +
- s"but $colName is of type $otherType"
- )
- }
-
- // if there are conflicting updates, throw an exception
- // there are two illegal scenarios:
- // - multiple updates to the same column
- // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
- case updates if hasExactMatch(updates, col, resolver) =>
- val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
- throw new AnalysisException(
- "Updates are in conflict for these columns: " +
- conflictingCols.distinct.mkString(", "))
- }
- }
- }
-
- private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
- val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
- Seq(Literal(field.name), expr)
- }
- CreateNamedStruct(namedStructExprs)
- }
-
- private def hasExactMatch(
- updates: Seq[ColumnUpdate],
- col: NamedExpression,
- resolver: Resolver): Boolean = {
-
- updates.exists(assignment => isExactMatch(assignment, col, resolver))
- }
-
- private def isExactMatch(
- update: ColumnUpdate,
- col: NamedExpression,
- resolver: Resolver): Boolean = {
-
- update.ref match {
- case Seq(namePart) if resolver(namePart, col.name) => true
- case _ => false
- }
- }
-
- protected def castIfNeeded(
- tableAttr: NamedExpression,
- expr: Expression,
- resolver: Resolver): Expression = {
-
- val storeAssignmentPolicy = conf.storeAssignmentPolicy
-
- // run the type check and catch type errors
- storeAssignmentPolicy match {
- case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
- if (expr.nullable && !tableAttr.nullable) {
- throw new AnalysisException(
- s"Cannot write nullable values to non-null column '${tableAttr.name}'")
- }
-
- // we use byName = true to catch cases when struct field names don't match
- // e.g. a struct with fields (a, b) is assigned as a struct with fields (a, c) or (b, a)
- val errors = new mutable.ArrayBuffer[String]()
- val canWrite = DataType.canWrite(
- expr.dataType, tableAttr.dataType, byName = true, resolver, tableAttr.name,
- storeAssignmentPolicy, err => errors += err)
-
- if (!canWrite) {
- throw new AnalysisException(s"Cannot write incompatible data:\n- ${errors.mkString("\n- ")}")
- }
-
- case _ => // OK
- }
-
- storeAssignmentPolicy match {
- case _ if tableAttr.dataType.sameType(expr.dataType) =>
- expr
- case StoreAssignmentPolicy.ANSI =>
- AnsiCast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
- case _ =>
- Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
- }
- }
-
- implicit protected def asAssignmentReference(expr: Expression): Seq[String] = expr match {
- case attr: AttributeReference => Seq(attr.name)
- case Alias(child, _) => asAssignmentReference(child)
- case GetStructField(child, _, Some(name)) => asAssignmentReference(child) :+ name
- case other: ExtractValue =>
- throw new AnalysisException(s"Updating nested fields is only supported for structs: $other")
- case other =>
- throw new AnalysisException(s"Cannot convert to a reference, unsupported expression: $other")
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala
deleted file mode 100644
index 7f0ca8fadded..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.Cast
-import org.apache.spark.sql.catalyst.plans.logical.Call
-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 @ Call(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 AnalysisException(
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala
deleted file mode 100644
index b50655d5c2b3..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis
-
-import java.util.Locale
-import org.apache.spark.sql.AnalysisException
-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.Call
-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.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)
- Call(procedure, args = buildArgExprs(normalizedParams, normalizedArgs))
- }
-
- 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 AnalysisException(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 AnalysisException(
- 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 AnalysisException(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 AnalysisException("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 AnalysisException(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 AnalysisException("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 AnalysisException(s"Cannot use catalog ${plugin.name}: not a ProcedureCatalog")
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RowLevelOperationsPredicateCheck.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RowLevelOperationsPredicateCheck.scala
deleted file mode 100644
index 0beab493f21e..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RowLevelOperationsPredicateCheck.scala
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.InSubquery
-import org.apache.spark.sql.catalyst.expressions.Not
-import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
-import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
-import org.apache.spark.sql.catalyst.plans.logical.InsertAction
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
-import org.apache.spark.sql.catalyst.plans.logical.UpdateAction
-import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-
-object RowLevelOperationsPredicateCheck extends (LogicalPlan => Unit) {
-
- override def apply(plan: LogicalPlan): Unit = {
- plan foreach {
- case DeleteFromTable(r, Some(condition)) if hasNullAwarePredicateWithinNot(condition) && isIcebergRelation(r) =>
- // this limitation is present since SPARK-25154 fix is not yet available
- // we use Not(EqualsNullSafe(cond, true)) when deciding which records to keep
- // such conditions are rewritten by Spark as an existential join and currently Spark
- // does not handle correctly NOT IN subqueries nested into other expressions
- failAnalysis("Null-aware predicate subqueries are not currently supported in DELETE")
-
- case UpdateTable(r, _, Some(condition)) if hasNullAwarePredicateWithinNot(condition) && isIcebergRelation(r) =>
- // this limitation is present since SPARK-25154 fix is not yet available
- // we use Not(EqualsNullSafe(cond, true)) when processing records that did not match
- // the update condition but were present in files we are overwriting
- // such conditions are rewritten by Spark as an existential join and currently Spark
- // does not handle correctly NOT IN subqueries nested into other expressions
- failAnalysis("Null-aware predicate subqueries are not currently supported in UPDATE")
-
- case merge: MergeIntoTable if isIcebergRelation(merge.targetTable) =>
- validateMergeIntoConditions(merge)
-
- case _ => // OK
- }
- }
-
- private def validateMergeIntoConditions(merge: MergeIntoTable): Unit = {
- checkMergeIntoCondition(merge.mergeCondition, "SEARCH")
- val actions = merge.matchedActions ++ merge.notMatchedActions
- actions.foreach {
- case DeleteAction(Some(cond)) => checkMergeIntoCondition(cond, "DELETE")
- case UpdateAction(Some(cond), _) => checkMergeIntoCondition(cond, "UPDATE")
- case InsertAction(Some(cond), _) => checkMergeIntoCondition(cond, "INSERT")
- case _ => // OK
- }
- }
-
- private def checkMergeIntoCondition(cond: Expression, condName: String): Unit = {
- // Spark already validates the conditions are deterministic and don't contain aggregates
- if (SubqueryExpression.hasSubquery(cond)) {
- throw new AnalysisException(
- s"Subqueries are not supported in conditions of MERGE operations. " +
- s"Found a subquery in the $condName condition: ${cond.sql}")
- }
- }
-
- private def hasNullAwarePredicateWithinNot(cond: Expression): Boolean = {
- cond.find {
- case Not(expr) if expr.find(_.isInstanceOf[InSubquery]).isDefined => true
- case _ => false
- }.isDefined
- }
-
- private def failAnalysis(msg: String): Unit = throw new AnalysisException(msg)
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AccumulateFiles.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AccumulateFiles.scala
deleted file mode 100644
index f673db3da216..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AccumulateFiles.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.expressions
-
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.catalyst.utils.SetAccumulator
-import org.apache.spark.sql.types.DataType
-import org.apache.spark.sql.types.IntegerType
-
-case class AccumulateFiles(
- filesAccumulator: SetAccumulator[String],
- child: Expression) extends UnaryExpression with CodegenFallback {
-
- override def dataType: DataType = IntegerType
- override def nullable: Boolean = true
- override def prettyName: String = "AccumulateFiles"
- override lazy val deterministic: Boolean = false
- private val RETURN_VAL: Integer = 1
-
- override def eval(input: InternalRow) : Any = {
- val resultVal = child.eval(input)
- filesAccumulator.add(resultVal.toString)
- RETURN_VAL
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeConditionsInRowLevelOperations.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeConditionsInRowLevelOperations.scala
deleted file mode 100644
index 621d0acd4e12..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeConditionsInRowLevelOperations.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.optimizer
-
-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.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
-
-// we have to optimize expressions used in delete/update before we can rewrite row-level operations
-// otherwise, we will have to deal with redundant casts and will not detect noop deletes
-// it is a temp solution since we cannot inject rewrite of row-level ops after operator optimizations
-object OptimizeConditionsInRowLevelOperations extends Rule[LogicalPlan] {
- override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
- case d @ DeleteFromTable(table, cond)
- if !SubqueryExpression.hasSubquery(cond.getOrElse(Literal.TrueLiteral)) && isIcebergRelation(table) =>
- val optimizedCond = optimizeCondition(cond.getOrElse(Literal.TrueLiteral), table)
- d.copy(condition = Some(optimizedCond))
- case u @ UpdateTable(table, _, cond)
- if !SubqueryExpression.hasSubquery(cond.getOrElse(Literal.TrueLiteral)) && isIcebergRelation(table) =>
- val optimizedCond = optimizeCondition(cond.getOrElse(Literal.TrueLiteral), table)
- u.copy(condition = Some(optimizedCond))
- }
-
- private def optimizeCondition(cond: Expression, table: LogicalPlan): Expression = {
- val optimizer = SparkSession.active.sessionState.optimizer
- optimizer.execute(Filter(cond, table)) match {
- case Filter(optimizedCondition, _) => optimizedCondition
- case _: LocalRelation => Literal.FalseLiteral
- case _: DataSourceV2ScanRelation => Literal.TrueLiteral
- case _ => cond
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesInRowLevelOperations.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesInRowLevelOperations.scala
deleted file mode 100644
index f0794d79c4a5..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesInRowLevelOperations.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-
-// a temp solution until PullupCorrelatedPredicates handles row-level operations in Spark
-object PullupCorrelatedPredicatesInRowLevelOperations extends Rule[LogicalPlan] {
- override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
- case d @ DeleteFromTable(table, Some(cond)) if SubqueryExpression.hasSubquery(cond) && isIcebergRelation(table) =>
- val transformedCond = transformCond(table, cond)
- d.copy(condition = Some(transformedCond))
-
- case u @ UpdateTable(table, _, Some(cond)) if SubqueryExpression.hasSubquery(cond) && isIcebergRelation(table) =>
- val transformedCond = transformCond(table, cond)
- u.copy(condition = Some(transformedCond))
- }
-
- // Spark pulls up correlated predicates only for UnaryNodes
- // DeleteFromTable and UpdateTable do not extend UnaryNode so they are ignored in that rule
- // We have this workaround until it is fixed in Spark
- private def transformCond(table: LogicalPlan, cond: Expression): Expression = {
- val filter = Filter(cond, table)
- val transformedFilter = PullupCorrelatedPredicates.apply(filter)
- transformedFilter.asInstanceOf[Filter].condition
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala
deleted file mode 100644
index c9b0633246fb..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.iceberg.DistributionMode
-import org.apache.iceberg.spark.Spark3Util
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.expressions.Ascending
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.catalyst.expressions.EqualNullSafe
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.expressions.Not
-import org.apache.spark.sql.catalyst.expressions.SortOrder
-import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-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.RepartitionByExpression
-import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
-import org.apache.spark.sql.catalyst.plans.logical.Sort
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
-import org.apache.spark.sql.connector.catalog.Table
-import org.apache.spark.sql.connector.iceberg.catalog.ExtendedSupportsDelete
-import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
-import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.BooleanType
-
-case class RewriteDelete(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper {
-
- import ExtendedDataSourceV2Implicits._
- import RewriteRowLevelOperationHelper._
-
- override def conf: SQLConf = SQLConf.get
-
- override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
- // don't rewrite deletes that can be answered by passing filters to deleteWhere in SupportsDelete
- case d @ DeleteFromTable(r: DataSourceV2Relation, Some(cond))
- if isMetadataDelete(r, cond) && isIcebergRelation(r) =>
- d
-
- // rewrite all operations that require reading the table to delete records
- case DeleteFromTable(r: DataSourceV2Relation, Some(cond)) if isIcebergRelation(r) =>
- // TODO: do a switch based on whether we get BatchWrite or DeltaBatchWrite
- val writeInfo = newWriteInfo(r.schema)
- val mergeBuilder = r.table.asMergeable.newMergeBuilder("delete", writeInfo)
-
- val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation)
- val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder)
-
- val remainingRowFilter = Not(EqualNullSafe(cond, Literal(true, BooleanType)))
- val remainingRowsPlan = Filter(remainingRowFilter, scanPlan)
-
- val mergeWrite = mergeBuilder.asWriteBuilder.buildForBatch()
- val writePlan = buildWritePlan(remainingRowsPlan, r.table, r.output)
- ReplaceData(r, mergeWrite, writePlan)
- }
-
- private def buildWritePlan(
- remainingRowsPlan: LogicalPlan,
- table: Table,
- output: Seq[AttributeReference]): LogicalPlan = {
-
- val fileNameCol = findOutputAttr(remainingRowsPlan.output, FILE_NAME_COL)
- val rowPosCol = findOutputAttr(remainingRowsPlan.output, ROW_POS_COL)
-
- val icebergTable = Spark3Util.toIcebergTable(table)
- val distributionMode = Spark3Util.distributionModeFor(icebergTable)
- val planWithDistribution = distributionMode match {
- case DistributionMode.NONE =>
- remainingRowsPlan
- case _ =>
- // apply hash partitioning by file if the distribution mode is hash or range
- RepartitionByExpression(Seq(fileNameCol), remainingRowsPlan, None)
- }
-
- val order = Seq(SortOrder(fileNameCol, Ascending), SortOrder(rowPosCol, Ascending))
- val sort = Sort(order, global = false, planWithDistribution)
- Project(output, sort)
- }
-
- private def isMetadataDelete(relation: DataSourceV2Relation, cond: Expression): Boolean = {
- relation.table match {
- case t: ExtendedSupportsDelete if !SubqueryExpression.hasSubquery(cond) =>
- val predicates = splitConjunctivePredicates(cond)
- val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, relation.output)
- val dataSourceFilters = toDataSourceFilters(normalizedPredicates)
- val allPredicatesTranslated = normalizedPredicates.size == dataSourceFilters.length
- allPredicatesTranslated && t.canDeleteWhere(dataSourceFilters)
- case _ => false
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
deleted file mode 100644
index 0bffbcea43fa..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.iceberg.TableProperties.MERGE_CARDINALITY_CHECK_ENABLED
-import org.apache.iceberg.TableProperties.MERGE_CARDINALITY_CHECK_ENABLED_DEFAULT
-import org.apache.iceberg.util.PropertyUtil
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.expressions.Alias
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.IsNotNull
-import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.plans.FullOuter
-import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.LeftAnti
-import org.apache.spark.sql.catalyst.plans.RightOuter
-import org.apache.spark.sql.catalyst.plans.logical.AppendData
-import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-import org.apache.spark.sql.catalyst.plans.logical.InsertAction
-import org.apache.spark.sql.catalyst.plans.logical.Join
-import org.apache.spark.sql.catalyst.plans.logical.JoinHint
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.MergeAction
-import org.apache.spark.sql.catalyst.plans.logical.MergeInto
-import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
-import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
-import org.apache.spark.sql.catalyst.plans.logical.Project
-import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
-import org.apache.spark.sql.catalyst.plans.logical.UpdateAction
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
-import org.apache.spark.sql.connector.catalog.Table
-import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
-import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.BooleanType
-
-case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper {
- import ExtendedDataSourceV2Implicits._
- import RewriteMergeInto._
-
- override def conf: SQLConf = SQLConf.get
-
- override def apply(plan: LogicalPlan): LogicalPlan = {
- plan transform {
- case MergeIntoTable(target: DataSourceV2Relation, source, cond, matchedActions, notMatchedActions)
- if matchedActions.isEmpty && notMatchedActions.size == 1 && isIcebergRelation(target) =>
-
- val targetTableScan = buildSimpleScanPlan(target, cond)
-
- // NOT MATCHED conditions may only refer to columns in source so we can push them down
- val insertAction = notMatchedActions.head.asInstanceOf[InsertAction]
- val filteredSource = insertAction.condition match {
- case Some(insertCond) => Filter(insertCond, source)
- case None => source
- }
-
- // when there are no matched actions, use a left anti join to remove any matching rows and rewrite to use
- // append instead of replace. only unmatched source rows are passed to the merge and actions are all inserts.
- val joinPlan = Join(filteredSource, targetTableScan, LeftAnti, Some(cond), JoinHint.NONE)
-
- val outputExprs = insertAction.assignments.map(_.value)
- val outputColNames = target.output.map(_.name)
- val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => Alias(expr, name)() }
- val mergePlan = Project(outputCols, joinPlan)
-
- val writePlan = buildWritePlan(mergePlan, target.table)
-
- AppendData.byPosition(target, writePlan, Map.empty)
-
- case MergeIntoTable(target: DataSourceV2Relation, source, cond, matchedActions, notMatchedActions)
- if matchedActions.isEmpty && isIcebergRelation(target) =>
-
- val targetTableScan = buildSimpleScanPlan(target, cond)
-
- // when there are no matched actions, use a left anti join to remove any matching rows and rewrite to use
- // append instead of replace. only unmatched source rows are passed to the merge and actions are all inserts.
- val joinPlan = Join(source, targetTableScan, LeftAnti, Some(cond), JoinHint.NONE)
-
- val mergeParams = MergeIntoParams(
- isSourceRowPresent = TRUE_LITERAL,
- isTargetRowPresent = FALSE_LITERAL,
- matchedConditions = Nil,
- matchedOutputs = Nil,
- notMatchedConditions = notMatchedActions.map(getClauseCondition),
- notMatchedOutputs = notMatchedActions.map(actionOutput),
- targetOutput = Nil,
- joinedAttributes = joinPlan.output
- )
-
- val mergePlan = MergeInto(mergeParams, target.output, joinPlan)
- val writePlan = buildWritePlan(mergePlan, target.table)
-
- AppendData.byPosition(target, writePlan, Map.empty)
-
- case MergeIntoTable(target: DataSourceV2Relation, source, cond, matchedActions, notMatchedActions)
- if notMatchedActions.isEmpty && isIcebergRelation(target) =>
-
- val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", newWriteInfo(target.schema))
-
- // rewrite the matched actions to ensure there is always an action to produce the output row
- val (matchedConditions, matchedOutputs) = rewriteMatchedActions(matchedActions, target.output)
-
- // when there are no not-matched actions, use a right outer join to ignore source rows that do not match, but
- // keep all unmatched target rows that must be preserved.
- val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)())
- val newSourceTableScan = Project(sourceTableProj, source)
- val targetTableScan = buildDynamicFilterTargetScan(mergeBuilder, target, source, cond, matchedActions)
- val joinPlan = Join(newSourceTableScan, targetTableScan, RightOuter, Some(cond), JoinHint.NONE)
-
- val mergeParams = MergeIntoParams(
- isSourceRowPresent = IsNotNull(findOutputAttr(joinPlan.output, ROW_FROM_SOURCE)),
- isTargetRowPresent = TRUE_LITERAL,
- matchedConditions = matchedConditions,
- matchedOutputs = matchedOutputs,
- notMatchedConditions = Nil,
- notMatchedOutputs = Nil,
- targetOutput = target.output,
- joinedAttributes = joinPlan.output
- )
- val mergePlan = MergeInto(mergeParams, target.output, joinPlan)
- val writePlan = buildWritePlan(mergePlan, target.table)
- val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
-
- ReplaceData(target, batchWrite, writePlan)
-
- case MergeIntoTable(target: DataSourceV2Relation, source, cond, matchedActions, notMatchedActions)
- if isIcebergRelation(target) =>
-
- val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", newWriteInfo(target.schema))
-
- // rewrite the matched actions to ensure there is always an action to produce the output row
- val (matchedConditions, matchedOutputs) = rewriteMatchedActions(matchedActions, target.output)
-
- // use a full outer join because there are both matched and not matched actions
- val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)())
- val newSourceTableScan = Project(sourceTableProj, source)
- val targetTableScan = buildDynamicFilterTargetScan(mergeBuilder, target, source, cond, matchedActions)
- val targetTableProj = targetTableScan.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_TARGET)())
- val newTargetTableScan = Project(targetTableProj, targetTableScan)
- val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
-
- val mergeParams = MergeIntoParams(
- isSourceRowPresent = IsNotNull(findOutputAttr(joinPlan.output, ROW_FROM_SOURCE)),
- isTargetRowPresent = IsNotNull(findOutputAttr(joinPlan.output, ROW_FROM_TARGET)),
- matchedConditions = matchedConditions,
- matchedOutputs = matchedOutputs,
- notMatchedConditions = notMatchedActions.map(getClauseCondition),
- notMatchedOutputs = notMatchedActions.map(actionOutput),
- targetOutput = target.output,
- joinedAttributes = joinPlan.output
- )
- val mergePlan = MergeInto(mergeParams, target.output, joinPlan)
- val writePlan = buildWritePlan(mergePlan, target.table)
- val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
-
- ReplaceData(target, batchWrite, writePlan)
- }
- }
-
- private def actionOutput(clause: MergeAction): Option[Seq[Expression]] = {
- clause match {
- case u: UpdateAction =>
- Some(u.assignments.map(_.value))
- case _: DeleteAction =>
- None
- case i: InsertAction =>
- Some(i.assignments.map(_.value))
- }
- }
-
- private def getClauseCondition(clause: MergeAction): Expression = {
- clause.condition.getOrElse(TRUE_LITERAL)
- }
-
- private def buildDynamicFilterTargetScan(
- mergeBuilder: MergeBuilder,
- target: DataSourceV2Relation,
- source: LogicalPlan,
- cond: Expression,
- matchedActions: Seq[MergeAction]): LogicalPlan = {
- // Construct the plan to prune target based on join condition between source and target.
- val table = target.table
- val output = target.output
- val matchingRowsPlanBuilder = rel => Join(source, rel, Inner, Some(cond), JoinHint.NONE)
- val runCardinalityCheck = isCardinalityCheckEnabled(table) && isCardinalityCheckNeeded(matchedActions)
- buildDynamicFilterScanPlan(spark, target, output, mergeBuilder, cond, matchingRowsPlanBuilder, runCardinalityCheck)
- }
-
- private def rewriteMatchedActions(
- matchedActions: Seq[MergeAction],
- targetOutput: Seq[Expression]): (Seq[Expression], Seq[Option[Seq[Expression]]]) = {
- val startMatchedConditions = matchedActions.map(getClauseCondition)
- val catchAllIndex = startMatchedConditions.indexWhere {
- case Literal(true, BooleanType) =>
- true
- case _ =>
- false
- }
-
- val outputs = matchedActions.map(actionOutput)
- if (catchAllIndex < 0) {
- // all of the actions have non-trivial conditions. add an action to emit the target row if no action matches
- (startMatchedConditions :+ TRUE_LITERAL, outputs :+ Some(targetOutput))
- } else {
- // one "catch all" action will always match, prune the actions after it
- (startMatchedConditions.take(catchAllIndex + 1), outputs.take(catchAllIndex + 1))
- }
- }
-
- private def isCardinalityCheckEnabled(table: Table): Boolean = {
- PropertyUtil.propertyAsBoolean(
- table.properties(),
- MERGE_CARDINALITY_CHECK_ENABLED,
- MERGE_CARDINALITY_CHECK_ENABLED_DEFAULT)
- }
-
- private def isCardinalityCheckNeeded(actions: Seq[MergeAction]): Boolean = {
- def hasUnconditionalDelete(action: Option[MergeAction]): Boolean = {
- action match {
- case Some(DeleteAction(None)) => true
- case _ => false
- }
- }
- !(actions.size == 1 && hasUnconditionalDelete(actions.headOption))
- }
-}
-
-object RewriteMergeInto {
- private final val ROW_FROM_SOURCE = "_row_from_source_"
- private final val ROW_FROM_TARGET = "_row_from_target_"
- private final val TRUE_LITERAL = Literal(true, BooleanType)
- private final val FALSE_LITERAL = Literal(false, BooleanType)
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala
deleted file mode 100644
index 55e8b47a8998..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.expressions.Alias
-import org.apache.spark.sql.catalyst.expressions.EqualNullSafe
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.If
-import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.expressions.Not
-import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.Assignment
-import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilter
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-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.ReplaceData
-import org.apache.spark.sql.catalyst.plans.logical.Union
-import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation
-import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
-import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.BooleanType
-
-case class RewriteUpdate(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper {
-
- import ExtendedDataSourceV2Implicits._
-
- override def conf: SQLConf = SQLConf.get
-
- // TODO: can we do any better for no-op updates? when conditions evaluate to false/true?
- override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
- case UpdateTable(r: DataSourceV2Relation, assignments, Some(cond))
- if isIcebergRelation(r) && SubqueryExpression.hasSubquery(cond) =>
-
- val writeInfo = newWriteInfo(r.schema)
- val mergeBuilder = r.table.asMergeable.newMergeBuilder("update", writeInfo)
-
- // since we are processing matched and not matched rows using separate jobs
- // there will be two scans but we want to execute the dynamic file filter only once
- // so the first job uses DynamicFileFilter and the second one uses the underlying scan plan
- // both jobs share the same SparkMergeScan instance to ensure they operate on same files
- val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation)
- val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder)
- val underlyingScanPlan = scanPlan match {
- case DynamicFileFilter(plan, _, _) => plan.clone()
- case _ => scanPlan.clone()
- }
-
- // build a plan for records that match the cond and should be updated
- val matchedRowsPlan = Filter(cond, scanPlan)
- val updatedRowsPlan = buildUpdateProjection(r, matchedRowsPlan, assignments)
-
- // build a plan for records that did not match the cond but had to be copied over
- val remainingRowFilter = Not(EqualNullSafe(cond, Literal(true, BooleanType)))
- val remainingRowsPlan = Filter(remainingRowFilter, Project(r.output, underlyingScanPlan))
-
- // new state is a union of updated and copied over records
- val updatePlan = Union(updatedRowsPlan, remainingRowsPlan)
-
- val mergeWrite = mergeBuilder.asWriteBuilder.buildForBatch()
- val writePlan = buildWritePlan(updatePlan, r.table)
- ReplaceData(r, mergeWrite, writePlan)
-
- case UpdateTable(r: DataSourceV2Relation, assignments, Some(cond)) if isIcebergRelation(r) =>
- val writeInfo = newWriteInfo(r.schema)
- val mergeBuilder = r.table.asMergeable.newMergeBuilder("update", writeInfo)
-
- val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation)
- val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder)
-
- val updateProjection = buildUpdateProjection(r, scanPlan, assignments, cond)
-
- val mergeWrite = mergeBuilder.asWriteBuilder.buildForBatch()
- val writePlan = buildWritePlan(updateProjection, r.table)
- ReplaceData(r, mergeWrite, writePlan)
- }
-
- private def buildUpdateProjection(
- relation: DataSourceV2Relation,
- scanPlan: LogicalPlan,
- assignments: Seq[Assignment],
- cond: Expression = Literal.TrueLiteral): LogicalPlan = {
-
- // this method relies on the fact that the assignments have been aligned before
- require(relation.output.size == assignments.size, "assignments must be aligned")
-
- // Spark is going to execute the condition for each column but it seems we cannot avoid this
- val assignedExprs = assignments.map(_.value)
- val updatedExprs = assignedExprs.zip(relation.output).map { case (assignedExpr, attr) =>
- // use semanticEquals to avoid unnecessary if expressions as we may run after operator optimization
- if (attr.semanticEquals(assignedExpr)) {
- attr
- } else if (cond == Literal.TrueLiteral) {
- Alias(assignedExpr, attr.name)()
- } else {
- val updatedExpr = If(cond, assignedExpr, attr)
- Alias(updatedExpr, attr.name)()
- }
- }
-
- Project(updatedExprs, scanPlan)
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
deleted file mode 100644
index 36e19650a5e4..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
+++ /dev/null
@@ -1,302 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.FunctionIdentifier
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.Expression
-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
-
-class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface {
-
- private lazy val substitutor = new VariableSubstitution()
- 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)
- }
-
- /**
- * 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 {
- 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("/\\*.*?\\*/", " ")
- .trim()
- normalized.startsWith("call") || (
- 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)))
- }
-
- private def isSnapshotRefDdl(normalized: String): Boolean = {
- normalized.contains("create branch") ||
- normalized.contains("replace branch") ||
- normalized.contains("drop branch") ||
- normalized.contains("create tag") ||
- normalized.contains("replace tag") ||
- 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)
- }
- }
-}
-
-/* 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)) =>
- 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)
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala
deleted file mode 100644
index 0786d586d880..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala
+++ /dev/null
@@ -1,378 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.AnalysisException
-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.collection.JavaConverters._
-
-class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergSqlExtensionsBaseVisitor[AnyRef] {
-
- /**
- * Create a [[CallStatement]] for a stored procedure call.
- */
- override def visitCall(ctx: CallContext): CallStatement = withOrigin(ctx) {
- val name = ctx.multipartIdentifier.parts.asScala.map(_.getText)
- val args = ctx.callArgument.asScala.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 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 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 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),
- ctx.fieldList.fields.asScala.map(_.getText))
- }
-
- /**
- * Create an DROP IDENTIFIER FIELDS logical command.
- */
- override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) {
- DropIdentifierFields(
- typedVisit[Seq[String]](ctx.multipartIdentifier),
- ctx.fieldList.fields.asScala.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 AnalysisException(
- "ALTER TABLE has no changes: missing both distribution and ordering clauses")
- }
-
- val distributionMode = if (distributionSpec != null) {
- DistributionMode.HASH
- } else if (orderingSpec.UNORDERED != null || orderingSpec.LOCALLY != null) {
- DistributionMode.NONE
- } else {
- DistributionMode.RANGE
- }
-
- val ordering = if (orderingSpec != null && orderingSpec.order != null) {
- orderingSpec.order.fields.asScala.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 AnalysisException("ALTER TABLE contains multiple distribution clauses")
- }
-
- if (writeSpec.writeOrderingSpec.size > 1) {
- throw new AnalysisException("ALTER TABLE contains multiple ordering clauses")
- }
-
- val distributionSpec = writeSpec.writeDistributionSpec.asScala.headOption.orNull
- val orderingSpec = writeSpec.writeOrderingSpec.asScala.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 = ctx.arguments.asScala.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) {
- ctx.parts.asScala.map(_.getText)
- }
-
- /**
- * 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 = {
- ctx.children.asScala.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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala
deleted file mode 100644
index ae5a2391aa8f..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 Command {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala
deleted file mode 100644
index 2041a983700a..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-case class BranchOptions(snapshotId: Option[Long], numSnapshots: Option[Long],
- snapshotRetain: Option[Long], snapshotRefRetain: Option[Long])
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala
deleted file mode 100644
index 033d3f567045..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.util.truncatedString
-import org.apache.spark.sql.connector.iceberg.catalog.Procedure
-import scala.collection.Seq
-
-case class Call(procedure: Procedure, args: Seq[Expression]) extends Command {
- override lazy val output: Seq[Attribute] = procedure.outputType.toAttributes
-
- override def simpleString(maxFields: Int): String = {
- s"Call${truncatedString(output, "[", ", ", "]", maxFields)} ${procedure.description}"
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala
deleted file mode 100644
index 06ee4cf1625c..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 Command {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala
deleted file mode 100644
index 1488bdbc8b13..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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 Command {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala
deleted file mode 100644
index 7d72a8ea0400..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.expressions.Attribute
-
-case class DropBranch(table: Seq[String], branch: String, ifExists: Boolean) extends Command {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala
deleted file mode 100644
index 115af1586d7a..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.expressions.Attribute
-
-case class DropIdentifierFields(
- table: Seq[String],
- fields: Seq[String]) extends Command {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala
deleted file mode 100644
index 5fe8f92b1b86..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 Command {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala
deleted file mode 100644
index 2ea43fd3e323..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.expressions.Attribute
-
-case class DropTag(table: Seq[String], tag: String, ifExists: Boolean) extends Command {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DynamicFileFilter.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DynamicFileFilter.scala
deleted file mode 100644
index be7059698cd3..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DynamicFileFilter.scala
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.AttributeSet
-import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.catalyst.utils.SetAccumulator
-import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
-
-// TODO: fix stats (ignore the fact it is a binary node and report only scanRelation stats)
-case class DynamicFileFilter(
- scanPlan: LogicalPlan,
- fileFilterPlan: LogicalPlan,
- filterable: SupportsFileFilter) extends BinaryNode {
-
- @transient
- override lazy val references: AttributeSet = AttributeSet(fileFilterPlan.output)
-
- override def left: LogicalPlan = scanPlan
- override def right: LogicalPlan = fileFilterPlan
- override def output: Seq[Attribute] = scanPlan.output
-
- override def simpleString(maxFields: Int): String = {
- s"DynamicFileFilter${truncatedString(output, "[", ", ", "]", maxFields)}"
- }
-}
-
-case class DynamicFileFilterWithCardinalityCheck(
- scanPlan: LogicalPlan,
- fileFilterPlan: LogicalPlan,
- filterable: SupportsFileFilter,
- filesAccumulator: SetAccumulator[String]) extends BinaryNode {
-
- @transient
- override lazy val references: AttributeSet = AttributeSet(fileFilterPlan.output)
-
- override def left: LogicalPlan = scanPlan
- override def right: LogicalPlan = fileFilterPlan
- override def output: Seq[Attribute] = scanPlan.output
-
- override def simpleString(maxFields: Int): String = {
- s"DynamicFileFilterWithCardinalityCheck${truncatedString(output, "[", ", ", "]", maxFields)}"
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.scala
deleted file mode 100644
index 04c3236bc1ad..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.Expression
-
-case class MergeInto(
- mergeIntoProcessor: MergeIntoParams,
- output: Seq[Attribute],
- child: LogicalPlan) extends UnaryNode
-
-case class MergeIntoParams(
- isSourceRowPresent: Expression,
- isTargetRowPresent: Expression,
- matchedConditions: Seq[Expression],
- matchedOutputs: Seq[Option[Seq[Expression]]],
- notMatchedConditions: Seq[Expression],
- notMatchedOutputs: Seq[Option[Seq[Expression]]],
- targetOutput: Seq[Expression],
- joinedAttributes: Seq[Attribute]) extends Serializable
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala
deleted file mode 100644
index 16fd559c05ab..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.plans.logical
-
-import org.apache.spark.sql.catalyst.analysis.NamedRelation
-import org.apache.spark.sql.connector.write.BatchWrite
-
-case class ReplaceData(
- table: NamedRelation,
- write: BatchWrite,
- query: LogicalPlan) extends V2WriteCommand {
-
- def isByName: Boolean = false
-
- def withNewQuery(newQuery: LogicalPlan): ReplaceData = copy(query = newQuery)
-
- def withNewTable(newTable: NamedRelation): ReplaceData = copy(table = newTable)
-
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala
deleted file mode 100644
index 3ad8c59bfc82..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 Command {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala
deleted file mode 100644
index 2e9a34b87204..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 Command {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala
deleted file mode 100644
index 6afe1478d747..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.spark.sql.catalyst.plans.logical
-
-case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long])
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
deleted file mode 100644
index 5842b094beaf..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 ParsedStatement
-
-/**
- * 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
deleted file mode 100644
index 743fce9d746a..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.spark.sql.catalyst.utils
-
-import java.util.UUID
-import org.apache.iceberg.spark.Spark3Util
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.analysis.Resolver
-import org.apache.spark.sql.catalyst.expressions.AccumulateFiles
-import org.apache.spark.sql.catalyst.expressions.Alias
-import org.apache.spark.sql.catalyst.expressions.And
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.catalyst.expressions.AttributeSet
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.GreaterThan
-import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.expressions.PredicateHelper
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
-import org.apache.spark.sql.catalyst.expressions.aggregate.Complete
-import org.apache.spark.sql.catalyst.expressions.aggregate.Sum
-import org.apache.spark.sql.catalyst.plans.logical.Aggregate
-import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilter
-import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilterWithCardinalityCheck
-import org.apache.spark.sql.catalyst.plans.logical.Filter
-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.Repartition
-import org.apache.spark.sql.connector.catalog.Table
-import org.apache.spark.sql.connector.iceberg.distributions.OrderedDistribution
-import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
-import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
-import org.apache.spark.sql.connector.read.ScanBuilder
-import org.apache.spark.sql.connector.write.LogicalWriteInfo
-import org.apache.spark.sql.connector.write.LogicalWriteInfoImpl
-import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
-import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits
-import org.apache.spark.sql.execution.datasources.v2.PushDownUtils
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.util.CaseInsensitiveStringMap
-
-trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
-
- import DataSourceV2Implicits._
- import RewriteRowLevelOperationHelper._
- import ExtendedDataSourceV2Implicits.ScanBuilderHelper
-
- protected def spark: SparkSession
- def conf: SQLConf
- protected lazy val resolver: Resolver = conf.resolver
-
- protected def buildSimpleScanPlan(
- relation: DataSourceV2Relation,
- cond: Expression): LogicalPlan = {
-
- val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
-
- pushFilters(scanBuilder, cond, relation.output)
-
- val scan = scanBuilder.asIceberg.withMetadataColumns(FILE_NAME_COL, ROW_POS_COL).build()
- val outputAttrs = toOutputAttrs(scan.readSchema(), relation.output)
- val predicates = extractFilters(cond, relation.output).reduceLeftOption(And)
- val scanRelation = DataSourceV2ScanRelation(relation, scan, outputAttrs)
-
- predicates.map(Filter(_, scanRelation)).getOrElse(scanRelation)
- }
-
- protected def buildDynamicFilterScanPlan(
- spark: SparkSession,
- relation: DataSourceV2Relation,
- tableAttrs: Seq[AttributeReference],
- mergeBuilder: MergeBuilder,
- cond: Expression,
- matchingRowsPlanBuilder: DataSourceV2ScanRelation => LogicalPlan,
- runCardinalityCheck: Boolean = false): LogicalPlan = {
-
- val scanBuilder = mergeBuilder.asScanBuilder
-
- pushFilters(scanBuilder, cond, tableAttrs)
-
- val scan = scanBuilder.asIceberg.withMetadataColumns(FILE_NAME_COL, ROW_POS_COL).build()
- val outputAttrs = toOutputAttrs(scan.readSchema(), tableAttrs)
- val scanRelation = DataSourceV2ScanRelation(relation, scan, outputAttrs)
-
- scan match {
- case filterable: SupportsFileFilter if runCardinalityCheck =>
- val affectedFilesAcc = new SetAccumulator[String]()
- spark.sparkContext.register(affectedFilesAcc, AFFECTED_FILES_ACC_NAME)
-
- val matchingRowsPlan = matchingRowsPlanBuilder(scanRelation)
- val matchingFilesPlan = buildFileFilterPlan(affectedFilesAcc, matchingRowsPlan)
-
- DynamicFileFilterWithCardinalityCheck(
- scanRelation,
- matchingFilesPlan,
- filterable,
- affectedFilesAcc)
-
- case filterable: SupportsFileFilter =>
- val matchingRowsPlan = matchingRowsPlanBuilder(scanRelation)
- val matchingFilesPlan = buildFileFilterPlan(scanRelation.output, matchingRowsPlan)
- DynamicFileFilter(scanRelation, matchingFilesPlan, filterable)
-
- case _ =>
- scanRelation
- }
- }
-
- private def extractFilters(cond: Expression, tableAttrs: Seq[AttributeReference]): Seq[Expression] = {
- val tableAttrSet = AttributeSet(tableAttrs)
- splitConjunctivePredicates(cond).filter(_.references.subsetOf(tableAttrSet))
- }
-
- private def pushFilters(
- scanBuilder: ScanBuilder,
- cond: Expression,
- tableAttrs: Seq[AttributeReference]): Unit = {
- val predicates = extractFilters(cond, tableAttrs)
- if (predicates.nonEmpty) {
- val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, tableAttrs)
- PushDownUtils.pushFilters(scanBuilder, normalizedPredicates)
- }
- }
-
- protected def toDataSourceFilters(predicates: Seq[Expression]): Array[sources.Filter] = {
- predicates.flatMap { p =>
- val translatedFilter = DataSourceStrategy.translateFilter(p, supportNestedPredicatePushdown = true)
- if (translatedFilter.isEmpty) {
- logWarning(s"Cannot translate expression to source filter: $p")
- }
- translatedFilter
- }.toArray
- }
-
- protected def newWriteInfo(schema: StructType): LogicalWriteInfo = {
- val uuid = UUID.randomUUID()
- LogicalWriteInfoImpl(queryId = uuid.toString, schema, CaseInsensitiveStringMap.empty)
- }
-
- private def buildFileFilterPlan(tableAttrs: Seq[AttributeReference], matchingRowsPlan: LogicalPlan): LogicalPlan = {
- val fileAttr = findOutputAttr(tableAttrs, FILE_NAME_COL)
- val agg = Aggregate(Seq(fileAttr), Seq(fileAttr), matchingRowsPlan)
- Project(Seq(findOutputAttr(agg.output, FILE_NAME_COL)), agg)
- }
-
- private def buildFileFilterPlan(
- filesAccumulator: SetAccumulator[String],
- prunedTargetPlan: LogicalPlan): LogicalPlan = {
- val fileAttr = findOutputAttr(prunedTargetPlan.output, FILE_NAME_COL)
- val rowPosAttr = findOutputAttr(prunedTargetPlan.output, ROW_POS_COL)
- val accumulatorExpr = Alias(AccumulateFiles(filesAccumulator, fileAttr), AFFECTED_FILES_ACC_ALIAS_NAME)()
- val projectList = Seq(fileAttr, rowPosAttr, accumulatorExpr)
- val projectPlan = Project(projectList, prunedTargetPlan)
- val affectedFilesAttr = findOutputAttr(projectPlan.output, AFFECTED_FILES_ACC_ALIAS_NAME)
- val aggSumCol = Alias(AggregateExpression(Sum(affectedFilesAttr), Complete, false), SUM_ROW_ID_ALIAS_NAME)()
- // Group by the rows by row id while collecting the files that need to be over written via accumulator.
- val aggPlan = Aggregate(Seq(fileAttr, rowPosAttr), Seq(aggSumCol), projectPlan)
- val sumAttr = findOutputAttr(aggPlan.output, SUM_ROW_ID_ALIAS_NAME)
- val havingExpr = GreaterThan(sumAttr, Literal(1L))
- // Identifies ambiguous row in the target.
- Filter(havingExpr, aggPlan)
- }
-
- protected def findOutputAttr(attrs: Seq[Attribute], attrName: String): Attribute = {
- attrs.find(attr => resolver(attr.name, attrName)).getOrElse {
- throw new AnalysisException(s"Cannot find $attrName in $attrs")
- }
- }
-
- protected def toOutputAttrs(schema: StructType, attrs: Seq[AttributeReference]): Seq[AttributeReference] = {
- val nameToAttr = attrs.map(_.name).zip(attrs).toMap
- schema.toAttributes.map {
- a => nameToAttr.get(a.name) match {
- case Some(ref) =>
- // keep the attribute id if it was present in the relation
- a.withExprId(ref.exprId)
- case _ =>
- // if the field is new, create a new attribute
- AttributeReference(a.name, a.dataType, a.nullable, a.metadata)()
- }
- }
- }
-
- protected def buildWritePlan(childPlan: LogicalPlan, table: Table): LogicalPlan = {
- val icebergTable = Spark3Util.toIcebergTable(table)
- val distribution = Spark3Util.buildRequiredDistribution(icebergTable)
- val ordering = Spark3Util.buildRequiredOrdering(distribution, icebergTable)
- // range partitioning in Spark triggers a skew estimation job prior to shuffling
- // we insert a round-robin partitioning to avoid executing the merge join twice
- val newChildPlan = distribution match {
- case _: OrderedDistribution =>
- val numShufflePartitions = conf.numShufflePartitions
- Repartition(numShufflePartitions, shuffle = true, childPlan)
- case _ =>
- childPlan
- }
- DistributionAndOrderingUtils.prepareQuery(distribution, ordering, newChildPlan, conf)
- }
-}
-
-object RewriteRowLevelOperationHelper {
- final val FILE_NAME_COL = "_file"
- final val ROW_POS_COL = "_pos"
-
- // `internal.metrics` prefix ensures the accumulator state is not tracked by Spark UI
- private final val AFFECTED_FILES_ACC_NAME = "internal.metrics.merge.affectedFiles"
- private final val AFFECTED_FILES_ACC_ALIAS_NAME = "_affectedFiles_"
- private final val SUM_ROW_ID_ALIAS_NAME = "_sum_"
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/SetAccumulator.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/SetAccumulator.scala
deleted file mode 100644
index a694666ec414..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/SetAccumulator.scala
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.catalyst.utils
-
-import java.util.Collections
-import org.apache.spark.util.AccumulatorV2
-
-class SetAccumulator[T] extends AccumulatorV2[T, java.util.Set[T]] {
- private val _set = Collections.synchronizedSet(new java.util.HashSet[T]())
-
- override def isZero: Boolean = _set.isEmpty
-
- override def copy(): AccumulatorV2[T, java.util.Set[T]] = {
- val newAcc = new SetAccumulator[T]()
- newAcc._set.addAll(_set)
- newAcc
- }
-
- override def reset(): Unit = _set.clear()
-
- override def add(v: T): Unit = _set.add(v)
-
- override def merge(other: AccumulatorV2[T, java.util.Set[T]]): Unit = {
- _set.addAll(other.value)
- }
-
- override def value: java.util.Set[T] = _set
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala
deleted file mode 100644
index b33de899174f..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala
deleted file mode 100644
index fcf609216379..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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
-
-case class CallExec(
- output: Seq[Attribute],
- procedure: Procedure,
- input: InternalRow) extends V2CommandExec {
-
- override protected def run(): Seq[InternalRow] = {
- procedure.call(input)
- }
-
- override def simpleString(maxFields: Int): String = {
- s"CallExec${truncatedString(output, "[", ", ", "]", maxFields)} ${procedure.description}"
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala
deleted file mode 100644
index 2ca586838c95..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala
deleted file mode 100644
index 77bda048c568..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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 V2CommandExec {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala
deleted file mode 100644
index f510d7e2d107..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala
deleted file mode 100644
index 525ed77437a5..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala
deleted file mode 100644
index 76128c5f47f5..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala
deleted file mode 100644
index 90ba3ff88276..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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 V2CommandExec {
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DynamicFileFilterExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DynamicFileFilterExec.scala
deleted file mode 100644
index 0ee403f1014e..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DynamicFileFilterExec.scala
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.execution.datasources.v2
-
-import org.apache.spark.SparkException
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.AttributeSet
-import org.apache.spark.sql.catalyst.expressions.SortOrder
-import org.apache.spark.sql.catalyst.plans.physical
-import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.catalyst.utils.SetAccumulator
-import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
-import org.apache.spark.sql.execution.BinaryExecNode
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.SQLExecution
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import scala.collection.JavaConverters._
-
-abstract class DynamicFileFilterExecBase(
- scanExec: SparkPlan,
- fileFilterExec: SparkPlan) extends BinaryExecNode {
-
- override lazy val metrics = Map(
- "candidateFiles" -> SQLMetrics.createMetric(sparkContext, "candidate files"),
- "matchingFiles" -> SQLMetrics.createMetric(sparkContext, "matching files"))
-
- @transient
- override lazy val references: AttributeSet = AttributeSet(fileFilterExec.output)
-
- override def left: SparkPlan = scanExec
- override def right: SparkPlan = fileFilterExec
- override def output: Seq[Attribute] = scanExec.output
- override def outputPartitioning: physical.Partitioning = scanExec.outputPartitioning
- override def outputOrdering: Seq[SortOrder] = scanExec.outputOrdering
- override def supportsColumnar: Boolean = scanExec.supportsColumnar
-
- /*
- If both target and source have the same partitioning we can have a problem here if our filter exec actually
- changes the output partitioning of the node. Currently this can only occur in the SinglePartition distribution is
- in use which only happens if both the target and source have a single partition, but if it does we have the potential
- of eliminating the only partition in the target. If there are no partitions in the target then we will throw an
- exception because the partitioning was assumed to be the same 1 partition in source and target. We fix this by making
- sure that we always return at least 1 empty partition, in the future we may need to handle more complicated
- partitioner scenarios.
- */
-
- override protected def doExecute(): RDD[InternalRow] = {
- val result = scanExec.execute()
- if (result.partitions.length == 0) {
- sparkContext.parallelize(Array.empty[InternalRow], 1)
- } else {
- result
- }
- }
- override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
- val result = scanExec.executeColumnar()
- if (result.partitions.length == 0) {
- sparkContext.parallelize(Array.empty[ColumnarBatch], 1)
- } else {
- result
- }
- }
-
- override def simpleString(maxFields: Int): String = {
- s"DynamicFileFilterExec${truncatedString(output, "[", ", ", "]", maxFields)}"
- }
-
- def postFileFilterMetric(candidateFiles: Int, matchingFiles: Int): Unit = {
- longMetric("candidateFiles").set(candidateFiles)
- longMetric("matchingFiles").set(matchingFiles)
- val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
- SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq)
- }
-}
-
-case class DynamicFileFilterExec(
- scanExec: SparkPlan,
- fileFilterExec: SparkPlan,
- @transient filterable: SupportsFileFilter)
- extends DynamicFileFilterExecBase(scanExec, fileFilterExec) {
-
- override protected def doPrepare(): Unit = {
- val rows = fileFilterExec.executeCollect()
- val matchedFileLocations = rows.map(_.getString(0))
- val metric = filterable.filterFiles(matchedFileLocations.toSet.asJava)
- postFileFilterMetric(metric.candidateFiles(), metric.matchingFiles())
- }
-}
-
-case class DynamicFileFilterWithCardinalityCheckExec(
- scanExec: SparkPlan,
- fileFilterExec: SparkPlan,
- @transient filterable: SupportsFileFilter,
- filesAccumulator: SetAccumulator[String])
- extends DynamicFileFilterExecBase(scanExec, fileFilterExec) {
-
- override protected def doPrepare(): Unit = {
- val rows = fileFilterExec.executeCollect()
- if (rows.length > 0) {
- throw new SparkException(
- "The ON search condition of the MERGE statement matched a single row from " +
- "the target table with multiple rows of the source table. This could result " +
- "in the target row being operated on more than once with an update or delete operation " +
- "and is not allowed.")
- }
- val matchedFileLocations = filesAccumulator.value
- val metric = filterable.filterFiles(matchedFileLocations)
- postFileFilterMetric(metric.candidateFiles(), metric.matchingFiles())
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedBatchScanExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedBatchScanExec.scala
deleted file mode 100644
index 83c9ca5815db..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedBatchScanExec.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.execution.datasources.v2
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.connector.read.InputPartition
-import org.apache.spark.sql.connector.read.PartitionReaderFactory
-import org.apache.spark.sql.connector.read.Scan
-
-// The only reason we need this class and cannot reuse BatchScanExec is because
-// BatchScanExec caches input partitions and we cannot apply file filtering before execution
-// Spark calls supportsColumnar during physical planning which, in turn, triggers split planning
-// We must ensure the result is not cached so that we can push down file filters later
-// The only difference compared to BatchScanExec is that we are using def instead of lazy val for splits
-case class ExtendedBatchScanExec(
- output: Seq[AttributeReference],
- @transient scan: Scan) extends DataSourceV2ScanExecBase {
-
- @transient private lazy val batch = scan.toBatch
-
- // TODO: unify the equal/hashCode implementation for all data source v2 query plans.
- override def equals(other: Any): Boolean = other match {
- case other: ExtendedBatchScanExec => this.batch == other.batch
- case _ => false
- }
-
- override def hashCode(): Int = batch.hashCode()
-
- override def partitions: Seq[InputPartition] = batch.planInputPartitions()
-
- override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory()
-
- override def inputRDD: RDD[InternalRow] = {
- new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar)
- }
-
- override def doCanonicalize(): ExtendedBatchScanExec = {
- this.copy(output = output.map(QueryPlan.normalizeExpressions(_, output)))
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala
deleted file mode 100644
index 3c1c44149b91..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.execution.datasources.v2
-
-import org.apache.iceberg.spark.source.SparkScanBuilder
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.connector.catalog.Table
-import org.apache.spark.sql.connector.iceberg.catalog.SupportsMerge
-import org.apache.spark.sql.connector.read.ScanBuilder
-
-// must be merged with DataSourceV2Implicits in Spark
-object ExtendedDataSourceV2Implicits {
- implicit class TableHelper(table: Table) {
- def asMergeable: SupportsMerge = {
- table match {
- case support: SupportsMerge =>
- support
- case _ =>
- throw new AnalysisException(s"Table does not support updates and deletes: ${table.name}")
- }
- }
- }
-
- implicit class ScanBuilderHelper(scanBuilder: ScanBuilder) {
- def asIceberg: SparkScanBuilder = {
- scanBuilder match {
- case iceberg: SparkScanBuilder =>
- iceberg
- case _ =>
- throw new AnalysisException(s"ScanBuilder is not from an Iceberg table: $scanBuilder")
- }
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
deleted file mode 100644
index 149df7069436..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.NamedRelation
-import org.apache.spark.sql.catalyst.expressions.And
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.catalyst.expressions.NamedExpression
-import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField
-import org.apache.spark.sql.catalyst.plans.logical.Call
-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.DynamicFileFilter
-import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilterWithCardinalityCheck
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.logical.MergeInto
-import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
-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.connector.catalog.Identifier
-import org.apache.spark.sql.connector.catalog.TableCatalog
-import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
-import org.apache.spark.sql.execution.FilterExec
-import org.apache.spark.sql.execution.LeafExecNode
-import org.apache.spark.sql.execution.ProjectExec
-import org.apache.spark.sql.execution.SparkPlan
-import scala.collection.JavaConverters._
-
-case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy {
-
- override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
- case c @ Call(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 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 CreateOrReplaceBranch(
- IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) =>
- CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil
-
- case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) =>
- DropBranchExec(catalog, ident, branch, ifExists) :: Nil
-
- case CreateOrReplaceTag(
- IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) =>
- CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil
-
- case DropTag(IcebergCatalogAndIdentifier(catalog, ident), tag, ifExists) =>
- DropTagExec(catalog, ident, tag, ifExists) :: 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 DynamicFileFilter(scanPlan, fileFilterPlan, filterable) =>
- DynamicFileFilterExec(planLater(scanPlan), planLater(fileFilterPlan), filterable) :: Nil
-
- case DynamicFileFilterWithCardinalityCheck(scanPlan, fileFilterPlan, filterable, filesAccumulator) =>
- DynamicFileFilterWithCardinalityCheckExec(
- planLater(scanPlan),
- planLater(fileFilterPlan),
- filterable,
- filesAccumulator) :: Nil
-
- case PhysicalOperation(project, filters, DataSourceV2ScanRelation(_, scan: SupportsFileFilter, output)) =>
- // projection and filters were already pushed down in the optimizer.
- // this uses PhysicalOperation to get the projection and ensure that if the batch scan does
- // not support columnar, a projection is added to convert the rows to UnsafeRow.
- val batchExec = ExtendedBatchScanExec(output, scan)
- withProjectAndFilter(project, filters, batchExec, !batchExec.supportsColumnar) :: Nil
-
- case ReplaceData(relation, batchWrite, query) =>
- ReplaceDataExec(batchWrite, refreshCache(relation), planLater(query)) :: Nil
-
- case MergeInto(mergeIntoParams, output, child) =>
- MergeIntoExec(mergeIntoParams, output, planLater(child)) :: 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 def withProjectAndFilter(
- project: Seq[NamedExpression],
- filters: Seq[Expression],
- scan: LeafExecNode,
- needsUnsafeConversion: Boolean): SparkPlan = {
- val filterCondition = filters.reduceLeftOption(And)
- val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)
-
- if (withFilter.output != project || needsUnsafeConversion) {
- ProjectExec(project, withFilter)
- } else {
- withFilter
- }
- }
-
- private def refreshCache(r: NamedRelation)(): Unit = {
- spark.sharedState.cacheManager.recacheByPlan(spark, r)
- }
-
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
deleted file mode 100644
index a7d8ad6ea813..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.execution.datasources.v2
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.BasePredicate
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
-import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
-import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.UnaryExecNode
-
-case class MergeIntoExec(
- mergeIntoParams: MergeIntoParams,
- output: Seq[Attribute],
- override val child: SparkPlan) extends UnaryExecNode {
-
- protected override def doExecute(): RDD[InternalRow] = {
- child.execute().mapPartitions {
- processPartition(mergeIntoParams, _)
- }
- }
-
- private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
- UnsafeProjection.create(exprs, attrs)
- }
-
- private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
- GeneratePredicate.generate(expr, attrs)
- }
-
- private def applyProjection(
- actions: Seq[(BasePredicate, Option[UnsafeProjection])],
- inputRow: InternalRow): InternalRow = {
-
- // Find the first combination where the predicate evaluates to true.
- // In case when there are overlapping condition in the MATCHED
- // clauses, for the first one that satisfies the predicate, the
- // corresponding action is applied. For example:
- // WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
- // WHEN MATCHED AND id = 5 OR id = 21 DELETE
- // In above case, when id = 5, it applies both that matched predicates. In this
- // case the first one we see is applied.
-
- val pair = actions.find {
- case (predicate, _) => predicate.eval(inputRow)
- }
-
- // Now apply the appropriate projection to produce an output row, or return null to suppress this row
- pair match {
- case Some((_, Some(projection))) =>
- projection.apply(inputRow)
- case _ =>
- null
- }
- }
-
- private def processPartition(
- params: MergeIntoParams,
- rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
-
- val joinedAttrs = params.joinedAttributes
- val isSourceRowPresentPred = generatePredicate(params.isSourceRowPresent, joinedAttrs)
- val isTargetRowPresentPred = generatePredicate(params.isTargetRowPresent, joinedAttrs)
- val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
- val matchedProjs = params.matchedOutputs.map(_.map(generateProjection(_, joinedAttrs)))
- val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
- val notMatchedProjs = params.notMatchedOutputs.map(_.map(generateProjection(_, joinedAttrs)))
- val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
- val nonMatchedPairs = notMatchedPreds zip notMatchedProjs
- val matchedPairs = matchedPreds zip matchedProjs
-
- /**
- * This method is responsible for processing a input row to emit the resultant row with an
- * additional column that indicates whether the row is going to be included in the final
- * output of merge or not.
- * 1. If there is a target row for which there is no corresponding source row (join condition not met)
- * - Only project the target columns with deleted flag set to false.
- * 2. If there is a source row for which there is no corresponding target row (join condition not met)
- * - Apply the not matched actions (i.e INSERT actions) if non match conditions are met.
- * 3. If there is a source row for which there is a corresponding target row (join condition met)
- * - Apply the matched actions (i.e DELETE or UPDATE actions) if match conditions are met.
- */
- def processRow(inputRow: InternalRow): InternalRow = {
- if (!isSourceRowPresentPred.eval(inputRow)) {
- projectTargetCols.apply(inputRow)
- } else if (!isTargetRowPresentPred.eval(inputRow)) {
- applyProjection(nonMatchedPairs, inputRow)
- } else {
- applyProjection(matchedPairs, inputRow)
- }
- }
-
- rowIterator
- .map(processRow)
- .filter(row => row != null)
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala
deleted file mode 100644
index f26a8c71b550..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.spark.sql.execution.datasources.v2
-
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.connector.write.BatchWrite
-import org.apache.spark.sql.execution.SparkPlan
-
-case class ReplaceDataExec(
- batchWrite: BatchWrite,
- refreshCache: () => Unit,
- query: SparkPlan) extends V2TableWriteExec {
-
- override protected def run(): Seq[InternalRow] = {
- // calling prepare() ensures we execute DynamicFileFilter if present
- prepare()
- val writtenRows = writeWithV2(batchWrite)
- refreshCache()
- writtenRows
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala
deleted file mode 100644
index 11e900c77b1e..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 V2CommandExec {
- 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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala
deleted file mode 100644
index 7fad2dc016d2..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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 SetIdentifierFieldsExec(
- catalog: TableCatalog,
- ident: Identifier,
- fields: Seq[String]) extends V2CommandExec {
- 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(scala.collection.JavaConverters.seqAsJavaList(fields))
- .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/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala
deleted file mode 100644
index 9916b535067c..000000000000
--- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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.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: DistributionMode,
- sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends V2CommandExec {
-
- 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()
- sortOrder.foreach {
- case (term, SortDirection.ASC, nullOrder) =>
- orderBuilder.asc(term, nullOrder)
- case (term, SortDirection.DESC, nullOrder) =>
- orderBuilder.desc(term, nullOrder)
- }
- orderBuilder.commit()
-
- txn.updateProperties()
- .set(WRITE_DISTRIBUTION_MODE, distributionMode.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/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java
deleted file mode 100644
index 8918dfec6584..000000000000
--- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
deleted file mode 100644
index 0a1cf7520463..000000000000
--- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg.spark.extensions;
-
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
-
-import java.util.Map;
-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.SparkCatalogTestBase;
-import org.apache.iceberg.spark.SparkTestBase;
-import org.apache.spark.sql.SparkSession;
-import org.apache.spark.sql.internal.SQLConf;
-import org.junit.BeforeClass;
-
-public abstract class SparkExtensionsTestBase extends SparkCatalogTestBase {
-
- public SparkExtensionsTestBase(
- String catalogName, String implementation, Map config) {
- super(catalogName, implementation, config);
- }
-
- @BeforeClass
- public static void startMetastoreAndSpark() {
- SparkTestBase.metastore = new TestHiveMetastore();
- metastore.start();
- SparkTestBase.hiveConf = metastore.hiveConf();
-
- SparkTestBase.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")
- .enableHiveSupport()
- .getOrCreate();
-
- SparkTestBase.catalog =
- (HiveCatalog)
- CatalogUtil.loadCatalog(
- HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
deleted file mode 100644
index 37f6dc37d580..000000000000
--- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg.spark.extensions;
-
-import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
-import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_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 java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.stream.Collectors;
-import org.apache.iceberg.Snapshot;
-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.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public abstract class SparkRowLevelOperationsTestBase extends SparkExtensionsTestBase {
-
- private static final Random RANDOM = ThreadLocalRandom.current();
-
- protected final String fileFormat;
- protected final boolean vectorized;
- protected final String distributionMode;
-
- public SparkRowLevelOperationsTestBase(
- String catalogName,
- String implementation,
- Map config,
- String fileFormat,
- boolean vectorized,
- String distributionMode) {
- super(catalogName, implementation, config);
- this.fileFormat = fileFormat;
- this.vectorized = vectorized;
- this.distributionMode = distributionMode;
- }
-
- @Parameters(
- name =
- "catalogName = {0}, implementation = {1}, config = {2},"
- + " format = {3}, vectorized = {4}, distributionMode = {5}")
- public static Object[][] parameters() {
- return new Object[][] {
- {
- "testhive",
- SparkCatalog.class.getName(),
- ImmutableMap.of(
- "type", "hive",
- "default-namespace", "default"),
- "orc",
- true,
- WRITE_DISTRIBUTION_MODE_NONE
- },
- {
- "testhadoop",
- SparkCatalog.class.getName(),
- ImmutableMap.of("type", "hadoop"),
- "parquet",
- RANDOM.nextBoolean(),
- WRITE_DISTRIBUTION_MODE_HASH
- },
- {
- "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
- ),
- "avro",
- false,
- WRITE_DISTRIBUTION_MODE_RANGE
- }
- };
- }
-
- protected abstract Map extraTableProperties();
-
- protected void initTable() {
- sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", tableName, DEFAULT_FILE_FORMAT, fileFormat);
- sql(
- "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')",
- tableName, WRITE_DISTRIBUTION_MODE, distributionMode);
-
- switch (fileFormat) {
- case "parquet":
- sql(
- "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')",
- tableName, PARQUET_VECTORIZATION_ENABLED, vectorized);
- break;
- case "orc":
- Assert.assertTrue(vectorized);
- break;
- case "avro":
- Assert.assertFalse(vectorized);
- 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) {
- sql("CREATE TABLE %s (%s) USING iceberg", tableName, schema);
- initTable();
-
- if (jsonData != null) {
- try {
- Dataset ds = toDS(schema, jsonData);
- ds.writeTo(tableName).append();
- } 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().length() > 0)
- .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 validateSnapshot(
- Snapshot snapshot,
- String operation,
- String changedPartitionCount,
- String deletedDataFiles,
- String addedDataFiles) {
- Assert.assertEquals("Operation must match", operation, snapshot.operation());
- Assert.assertEquals(
- "Changed partitions count must match",
- changedPartitionCount,
- snapshot.summary().get("changed-partition-count"));
- Assert.assertEquals(
- "Deleted data files count must match",
- deletedDataFiles,
- snapshot.summary().get("deleted-data-files"));
- Assert.assertEquals(
- "Added data files count must match",
- addedDataFiles,
- snapshot.summary().get("added-data-files"));
- }
-
- protected void sleep(long millis) {
- try {
- Thread.sleep(millis);
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
-}
diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
deleted file mode 100644
index 46963e42ffed..000000000000
--- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
+++ /dev/null
@@ -1,1090 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg.spark.extensions;
-
-import static org.apache.iceberg.types.Types.NestedField.optional;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-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.AssertHelpers;
-import org.apache.iceberg.DataFile;
-import org.apache.iceberg.Files;
-import org.apache.iceberg.MetadataTableType;
-import org.apache.iceberg.MetricsConfig;
-import org.apache.iceberg.TableProperties;
-import org.apache.iceberg.data.Record;
-import org.apache.iceberg.data.orc.GenericOrcWriter;
-import org.apache.iceberg.io.FileAppender;
-import org.apache.iceberg.io.OutputFile;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
-import org.apache.iceberg.types.Types;
-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.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-public class TestAddFilesProcedure extends SparkExtensionsTestBase {
-
- private final String sourceTableName = "source_table";
- private File fileTableDir;
-
- public TestAddFilesProcedure(
- String catalogName, String implementation, Map config) {
- super(catalogName, implementation, config);
- }
-
- @Rule public TemporaryFolder temp = new TemporaryFolder();
-
- @Before
- public void setupTempDirs() {
- try {
- fileTableDir = temp.newFolder();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @After
- public void dropTables() {
- sql("DROP TABLE IF EXISTS %s", sourceTableName);
- sql("DROP TABLE IF EXISTS %s", tableName);
- }
-
- @Test
- public void addDataUnpartitioned() {
- createUnpartitionedFileTable("parquet");
-
- String createIceberg =
- "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
-
- sql(createIceberg, tableName);
-
- Object result =
- scalarSql(
- "CALL %s.system.add_files('%s', '`parquet`.`%s`')",
- catalogName, tableName, fileTableDir.getAbsolutePath());
-
- Assert.assertEquals(2L, result);
-
- assertEquals(
- "Iceberg table contains correct data",
- sql("SELECT * FROM %s ORDER BY id", sourceTableName),
- sql("SELECT * FROM %s ORDER BY id", tableName));
- }
-
- @Ignore // TODO Classpath issues prevent us from actually writing to a Spark ORC table
- public void addDataUnpartitionedOrc() {
- createUnpartitionedFileTable("orc");
-
- String createIceberg =
- "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
-
- sql(createIceberg, tableName);
-
- Object result =
- scalarSql(
- "CALL %s.system.add_files('%s', '`orc`.`%s`')",
- catalogName, tableName, fileTableDir.getAbsolutePath());
-
- Assert.assertEquals(2L, result);
-
- assertEquals(
- "Iceberg table contains correct data",
- sql("SELECT * FROM %s ORDER BY id", sourceTableName),
- sql("SELECT * FROM %s ORDER BY id", tableName));
- }
-
- @Test
- public void addAvroFile() throws Exception {
- // Spark Session Catalog cannot load metadata tables
- // with "The namespace in session catalog must have exactly one name part"
- Assume.assumeFalse(catalogName.equals("spark_catalog"));
-
- // Create an Avro file
-
- Schema schema =
- SchemaBuilder.record("record")
- .fields()
- .requiredInt("id")
- .requiredString("data")
- .endRecord();
- GenericRecord record1 = new GenericData.Record(schema);
- record1.put("id", 1L);
- record1.put("data", "a");
- GenericRecord record2 = new GenericData.Record(schema);
- record2.put("id", 2L);
- record2.put("data", "b");
- File outputFile = temp.newFile("test.avro");
-
- DatumWriter datumWriter = new GenericDatumWriter(schema);
- DataFileWriter dataFileWriter = new DataFileWriter(datumWriter);
- dataFileWriter.create(schema, outputFile);
- dataFileWriter.append(record1);
- dataFileWriter.append(record2);
- dataFileWriter.close();
-
- String createIceberg = "CREATE TABLE %s (id Long, data String) USING iceberg";
- sql(createIceberg, tableName);
-
- Object result =
- scalarSql(
- "CALL %s.system.add_files('%s', '`avro`.`%s`')",
- catalogName, tableName, outputFile.getPath());
- Assert.assertEquals(1L, result);
-
- List