You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2023/01/12 15:43:56 UTC

[GitHub] [iceberg] nastra commented on a diff in pull request #6449: Delta: Support Snapshot Delta Lake Table to Iceberg Table

nastra commented on code in PR #6449:
URL: https://github.com/apache/iceberg/pull/6449#discussion_r1068240525


##########
delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+@SuppressWarnings("VisibilityModifier")
+public abstract class SparkDeltaLakeSnapshotTestBase {
+  protected static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore();
+    metastore.start();
+    SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf();
+
+    SparkDeltaLakeSnapshotTestBase.spark =
+        SparkSession.builder()
+            .master("local[2]")
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config(
+                "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname,
+                hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+            .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            // Needed for Delta Lake tests
+            .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    SparkDeltaLakeSnapshotTestBase.catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    SparkDeltaLakeSnapshotTestBase.catalog = null;
+    metastore.stop();

Review Comment:
   #5952 added some null checks around this, so I think it would be good to do the same here



##########
build.gradle:
##########
@@ -438,6 +442,76 @@ project(':iceberg-aws') {
   }
 }
 
+project(':iceberg-delta-lake') {
+  // use integration test since we can take advantages of spark 3.3 to read datafiles of delta lake table
+  // and create some tests involving sql query.
+  configurations {
+    integrationImplementation.extendsFrom testImplementation
+    integrationRuntime.extendsFrom testRuntimeOnly
+  }
+
+  dependencies {
+    implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
+    api project(':iceberg-api')
+    implementation project(':iceberg-common')
+    implementation project(':iceberg-core')
+    implementation project(':iceberg-parquet')
+    implementation project(':iceberg-orc')
+    implementation "com.fasterxml.jackson.core:jackson-databind"
+
+    compileOnly "io.delta:delta-standalone_${scalaVersion}"
+
+    compileOnly("org.apache.hadoop:hadoop-common") {
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+      exclude group: 'javax.servlet', module: 'servlet-api'
+      exclude group: 'com.google.code.gson', module: 'gson'
+    }
+
+    // The newest version of delta-core uses Spark 3.3.*. Since its only for test, we do
+    // not need to include older version of delta-core
+    if (sparkVersions.contains("3.3")) {
+      integrationImplementation "io.delta:delta-core_${scalaVersion}"
+      integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.3_${scalaVersion}")
+      integrationImplementation("org.apache.hadoop:hadoop-minicluster") {
+        exclude group: 'org.apache.avro', module: 'avro'
+        // to make sure netty libs only come from project(':iceberg-arrow')
+        exclude group: 'io.netty', module: 'netty-buffer'
+        exclude group: 'io.netty', module: 'netty-common'
+      }
+      integrationImplementation project(path: ':iceberg-hive-metastore')
+      integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
+      integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:3.3.1") {
+        exclude group: 'org.apache.avro', module: 'avro'
+        exclude group: 'org.apache.arrow'
+        exclude group: 'org.apache.parquet'
+        // to make sure netty libs only come from project(':iceberg-arrow')
+        exclude group: 'io.netty', module: 'netty-buffer'
+        exclude group: 'io.netty', module: 'netty-common'
+        exclude group: 'org.roaringbitmap'
+      }
+    }
+  }
+
+  // The newest version of delta-core uses Spark 3.3.*. The integration test should only be built
+  // if iceberg-spark-3.3 is available
+  if (sparkVersions.contains("3.3")) {
+    sourceSets {
+      integration {
+        java.srcDir "$projectDir/src/integration/java"
+        resources.srcDir "$projectDir/src/integration/resources"
+        compileClasspath += main.output + test.output
+        runtimeClasspath += main.output + test.output
+      }
+    }
+
+    task integrationTest(type: Test) {
+      testClassesDirs = sourceSets.integration.output.classesDirs
+      classpath = sourceSets.integration.runtimeClasspath
+    }
+  }

Review Comment:
   I believe this requires `check.dependsOn integrationTest` because otherwise the integration tests are not part of normal CI runs when e.g. `./gradlew check` is executed



##########
delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestBaseSnapshotDeltaLakeTableAction {
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  private String sourceTableLocation;
+  private final Configuration testHadoopConf = new Configuration();
+  private String newTableLocation;
+  private final Catalog testCatalog = new TestCatalog();
+
+  @Before
+  public void before() {
+    try {
+      File sourceFolder = temp1.newFolder();
+      File destFolder = temp2.newFolder();
+      sourceTableLocation = sourceFolder.toURI().toString();
+      newTableLocation = destFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testRequiredTableIdentifier() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .icebergCatalog(testCatalog)
+            .deltaLakeConfiguration(testHadoopConf)
+            .tableLocation(newTableLocation);
+    Assert.assertThrows(

Review Comment:
   given that we're introducing new code, it would be great to rather use AssertJ assertions here. So this code would look something like:
   
   ```
   Assertions.assertThatThrownBy(() -> testAction::execute)
       .isInstanceOf(IllegalArgumentException.class)
       .hasMessage(...)
   ```
   Also note that checks like that should make sure that the correct error msg comes back. Same for all the other methods



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(

Review Comment:
   ```suggestion
       Assertions.assertThat(icebergTableContents).containsAll(deltaTableContents);
   ```
   



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        icebergTableContents.containsAll(deltaTableContents));
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        deltaTableContents.containsAll(icebergTableContents));
+    Assert.assertEquals(
+        "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table",
+        deltaLog.update().getAllFiles().size(),
+        snapshotReport.snapshotDataFilesCount());
+  }
+
+  private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    Assert.assertEquals(
+        "The iceberg table should have the expected location",
+        expectedLocation,
+        icebergTable.location());
+  }
+
+  private void checkIcebergTableProperties(
+      String icebergTableIdentifier,
+      Map<String, String> expectedAdditionalProperties,
+      String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    ImmutableMap.Builder<String, String> expectedPropertiesBuilder = ImmutableMap.builder();
+    // The snapshot action will put some fixed properties to the table
+    expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE);
+    expectedPropertiesBuilder.putAll(expectedAdditionalProperties);
+    ImmutableMap<String, String> expectedProperties = expectedPropertiesBuilder.build();
+    Assert.assertTrue(

Review Comment:
   same as above (AssertJ check)



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+@SuppressWarnings("VisibilityModifier")
+public abstract class SparkDeltaLakeSnapshotTestBase {
+  protected static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore();
+    metastore.start();
+    SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf();
+
+    SparkDeltaLakeSnapshotTestBase.spark =
+        SparkSession.builder()
+            .master("local[2]")
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config(
+                "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname,
+                hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+            .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            // Needed for Delta Lake tests
+            .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    SparkDeltaLakeSnapshotTestBase.catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    SparkDeltaLakeSnapshotTestBase.catalog = null;
+    metastore.stop();
+    SparkDeltaLakeSnapshotTestBase.metastore = null;
+    spark.stop();
+    SparkDeltaLakeSnapshotTestBase.spark = null;
+  }
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  protected final String catalogName;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table");
+  protected final String tableName;
+
+  public SparkDeltaLakeSnapshotTestBase() {

Review Comment:
   just curious, is this being used?



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        icebergTableContents.containsAll(deltaTableContents));
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        deltaTableContents.containsAll(icebergTableContents));
+    Assert.assertEquals(
+        "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table",
+        deltaLog.update().getAllFiles().size(),
+        snapshotReport.snapshotDataFilesCount());
+  }
+
+  private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    Assert.assertEquals(
+        "The iceberg table should have the expected location",
+        expectedLocation,
+        icebergTable.location());
+  }
+
+  private void checkIcebergTableProperties(
+      String icebergTableIdentifier,
+      Map<String, String> expectedAdditionalProperties,
+      String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    ImmutableMap.Builder<String, String> expectedPropertiesBuilder = ImmutableMap.builder();
+    // The snapshot action will put some fixed properties to the table
+    expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE);
+    expectedPropertiesBuilder.putAll(expectedAdditionalProperties);
+    ImmutableMap<String, String> expectedProperties = expectedPropertiesBuilder.build();
+    Assert.assertTrue(
+        "The snapshot iceberg table should have the expected properties, all in original delta lake table, added by the action and user added ones",
+        icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet()));
+    Assert.assertTrue(
+        "The snapshot iceberg table's property should contains the original location",
+        icebergTable.properties().containsKey(ORIGINAL_LOCATION_PROP)
+            && icebergTable.properties().get(ORIGINAL_LOCATION_PROP).equals(deltaTableLocation));
+  }
+
+  private void checkDataFilePathsIntegrity(
+      String icebergTableIdentifier, String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+    // checkSnapshotIntegrity already checks the number of data files in the snapshot iceberg table
+    // equals that in the original delta lake table
+    List<String> deltaTableDataFilePaths =
+        deltaLog.update().getAllFiles().stream()
+            .map(f -> getFullFilePath(f.getPath(), deltaLog.getPath().toString()))
+            .collect(Collectors.toList());
+    icebergTable
+        .currentSnapshot()
+        .addedDataFiles(icebergTable.io())
+        .forEach(
+            dataFile -> {
+              Assert.assertTrue(

Review Comment:
   same as above (AssertJ)



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+@SuppressWarnings("VisibilityModifier")
+public abstract class SparkDeltaLakeSnapshotTestBase {
+  protected static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore();
+    metastore.start();
+    SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf();
+
+    SparkDeltaLakeSnapshotTestBase.spark =
+        SparkSession.builder()
+            .master("local[2]")
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config(
+                "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname,
+                hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+            .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            // Needed for Delta Lake tests
+            .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    SparkDeltaLakeSnapshotTestBase.catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    SparkDeltaLakeSnapshotTestBase.catalog = null;
+    metastore.stop();
+    SparkDeltaLakeSnapshotTestBase.metastore = null;
+    spark.stop();
+    SparkDeltaLakeSnapshotTestBase.spark = null;
+  }
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  protected final String catalogName;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table");
+  protected final String tableName;
+
+  public SparkDeltaLakeSnapshotTestBase() {
+    this(SparkCatalogConfig.HADOOP);
+  }
+
+  public SparkDeltaLakeSnapshotTestBase(SparkCatalogConfig config) {
+    this(config.catalogName(), config.implementation(), config.properties());
+  }
+
+  public SparkDeltaLakeSnapshotTestBase(
+      String catalogName, String implementation, Map<String, String> config) {
+    this.catalogName = catalogName;
+    this.validationCatalog =
+        catalogName.equals("testhadoop")
+            ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse)
+            : catalog;
+    this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog;
+
+    spark.conf().set("spark.sql.catalog." + catalogName, implementation);
+    config.forEach(
+        (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value));
+
+    if (config.get("type").equalsIgnoreCase("hadoop")) {
+      spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse);
+    }
+
+    this.tableName =
+        (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table";
+
+    sql("CREATE NAMESPACE IF NOT EXISTS default");
+  }
+
+  protected String tableName(String name) {

Review Comment:
   unused



##########
delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestBaseSnapshotDeltaLakeTableAction {
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  private String sourceTableLocation;
+  private final Configuration testHadoopConf = new Configuration();
+  private String newTableLocation;
+  private final Catalog testCatalog = new TestCatalog();
+
+  @Before
+  public void before() {
+    try {
+      File sourceFolder = temp1.newFolder();
+      File destFolder = temp2.newFolder();
+      sourceTableLocation = sourceFolder.toURI().toString();
+      newTableLocation = destFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testRequiredTableIdentifier() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .icebergCatalog(testCatalog)
+            .deltaLakeConfiguration(testHadoopConf)
+            .tableLocation(newTableLocation);
+    Assert.assertThrows(
+        "Should throw IllegalArgumentException if table identifier is not set",
+        IllegalArgumentException.class,
+        testAction::execute);
+  }
+
+  @Test
+  public void testRequiredIcebergCatalog() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .as(TableIdentifier.of("test", "test"))
+            .deltaLakeConfiguration(testHadoopConf)
+            .tableLocation(newTableLocation);
+
+    Assert.assertThrows(

Review Comment:
   ```suggestion
       Assertions.assertThatThrownBy(testAction::execute)
           .isInstanceOf(IllegalArgumentException.class)
           .hasMessage(
               "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier.");
   ```



##########
delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestBaseSnapshotDeltaLakeTableAction {
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  private String sourceTableLocation;
+  private final Configuration testHadoopConf = new Configuration();
+  private String newTableLocation;
+  private final Catalog testCatalog = new TestCatalog();
+
+  @Before
+  public void before() {
+    try {

Review Comment:
   nit: is the try-catch really needed here? Why not just add a `throws` to the method?



##########
delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.types.ArrayType;
+import io.delta.standalone.types.BinaryType;
+import io.delta.standalone.types.BooleanType;
+import io.delta.standalone.types.DoubleType;
+import io.delta.standalone.types.LongType;
+import io.delta.standalone.types.MapType;
+import io.delta.standalone.types.StringType;
+import io.delta.standalone.types.StructType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestDeltaLakeTypeToType {
+  private static final String optionalBooleanType = "testNullableBoolType";
+
+  private static final String requiredBinaryType = "testRequiredBinaryType";
+
+  private static final String doubleArrayType = "testNullableArrayType";
+
+  private static final String innerAtomicSchema = "testInnerAtomicSchema";
+
+  private static final String stringLongMapType = "testStringLongMap";
+
+  private StructType deltaAtomicSchema;
+
+  private StructType deltaNestedSchema;
+
+  @Before
+  public void constructDeltaLakeSchema() {
+    deltaAtomicSchema =
+        new StructType()
+            .add(optionalBooleanType, new BooleanType())
+            .add(requiredBinaryType, new BinaryType(), false);
+    deltaNestedSchema =
+        new StructType()
+            .add(innerAtomicSchema, deltaAtomicSchema)
+            .add(doubleArrayType, new ArrayType(new DoubleType(), true), false)
+            .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false);
+  }
+
+  @Test
+  public void testAtomicTypeConversion() {
+    Type converted =
+        DeltaLakeDataTypeVisitor.visit(
+            deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema));
+    Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields());
+    Assert.assertTrue(
+        "The BooleanType should be converted to BooleanType",
+        convertedSchema.findType(optionalBooleanType) instanceof Types.BooleanType);

Review Comment:
   when this test fails then the output will be `true != ..`, therefore it would be better to change this to `Assertions.assertThat(convertedSchema.findType(optionalBooleanType)).isInstanceOf(Types.BooleanType.class);`. That will show the class of the returned type in case the assertion ever fails.
   
   That being said, I'd probably change these 4 assertions to
   ```
   Assertions.assertThat(convertedSchema.findType(optionalBooleanType)).isInstanceOf(Types.BooleanType.class);
   Assertions.assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue();
   Assertions.assertThat(convertedSchema.findType(requiredBinaryType)).isInstanceOf(Types.BooleanType.class);
   Assertions.assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue();
   ```
   
   Same for all the other methods



##########
delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestBaseSnapshotDeltaLakeTableAction {
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  private String sourceTableLocation;
+  private final Configuration testHadoopConf = new Configuration();
+  private String newTableLocation;
+  private final Catalog testCatalog = new TestCatalog();
+
+  @Before
+  public void before() {
+    try {
+      File sourceFolder = temp1.newFolder();
+      File destFolder = temp2.newFolder();
+      sourceTableLocation = sourceFolder.toURI().toString();
+      newTableLocation = destFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testRequiredTableIdentifier() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .icebergCatalog(testCatalog)
+            .deltaLakeConfiguration(testHadoopConf)
+            .tableLocation(newTableLocation);
+    Assert.assertThrows(
+        "Should throw IllegalArgumentException if table identifier is not set",
+        IllegalArgumentException.class,
+        testAction::execute);
+  }
+
+  @Test
+  public void testRequiredIcebergCatalog() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .as(TableIdentifier.of("test", "test"))
+            .deltaLakeConfiguration(testHadoopConf)
+            .tableLocation(newTableLocation);
+
+    Assert.assertThrows(
+        "Should throw IllegalArgumentException if iceberg catalog is not set",
+        IllegalArgumentException.class,
+        testAction::execute);
+  }
+
+  @Test
+  public void testRequiredDeltaLakeConfiguration() {
+    SnapshotDeltaLakeTable testAction =
+        new BaseSnapshotDeltaLakeTableAction(sourceTableLocation)
+            .as(TableIdentifier.of("test", "test"))
+            .icebergCatalog(testCatalog)
+            .tableLocation(newTableLocation);
+    Assert.assertThrows(

Review Comment:
   same as above + needs a check for the right error message



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        icebergTableContents.containsAll(deltaTableContents));
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        deltaTableContents.containsAll(icebergTableContents));
+    Assert.assertEquals(
+        "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table",
+        deltaLog.update().getAllFiles().size(),
+        snapshotReport.snapshotDataFilesCount());
+  }
+
+  private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    Assert.assertEquals(
+        "The iceberg table should have the expected location",
+        expectedLocation,
+        icebergTable.location());
+  }
+
+  private void checkIcebergTableProperties(
+      String icebergTableIdentifier,
+      Map<String, String> expectedAdditionalProperties,
+      String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    ImmutableMap.Builder<String, String> expectedPropertiesBuilder = ImmutableMap.builder();
+    // The snapshot action will put some fixed properties to the table
+    expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE);
+    expectedPropertiesBuilder.putAll(expectedAdditionalProperties);
+    ImmutableMap<String, String> expectedProperties = expectedPropertiesBuilder.build();
+    Assert.assertTrue(
+        "The snapshot iceberg table should have the expected properties, all in original delta lake table, added by the action and user added ones",
+        icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet()));
+    Assert.assertTrue(

Review Comment:
   ```suggestion
       Assertions.assertThat(icebergTable.properties()).containsEntry(ORIGINAL_LOCATION_PROP, deltaTableLocation);
   ```



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        icebergTableContents.containsAll(deltaTableContents));
+    Assert.assertTrue(
+        "The original table and the transformed one should have the same contents",
+        deltaTableContents.containsAll(icebergTableContents));
+    Assert.assertEquals(
+        "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table",
+        deltaLog.update().getAllFiles().size(),
+        snapshotReport.snapshotDataFilesCount());
+  }
+
+  private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    Assert.assertEquals(
+        "The iceberg table should have the expected location",
+        expectedLocation,
+        icebergTable.location());
+  }
+
+  private void checkIcebergTableProperties(
+      String icebergTableIdentifier,
+      Map<String, String> expectedAdditionalProperties,
+      String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    ImmutableMap.Builder<String, String> expectedPropertiesBuilder = ImmutableMap.builder();
+    // The snapshot action will put some fixed properties to the table
+    expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE);
+    expectedPropertiesBuilder.putAll(expectedAdditionalProperties);
+    ImmutableMap<String, String> expectedProperties = expectedPropertiesBuilder.build();
+    Assert.assertTrue(
+        "The snapshot iceberg table should have the expected properties, all in original delta lake table, added by the action and user added ones",
+        icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet()));
+    Assert.assertTrue(
+        "The snapshot iceberg table's property should contains the original location",
+        icebergTable.properties().containsKey(ORIGINAL_LOCATION_PROP)
+            && icebergTable.properties().get(ORIGINAL_LOCATION_PROP).equals(deltaTableLocation));
+  }
+
+  private void checkDataFilePathsIntegrity(
+      String icebergTableIdentifier, String deltaTableLocation) {
+    Table icebergTable = getIcebergTable(icebergTableIdentifier);
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+    // checkSnapshotIntegrity already checks the number of data files in the snapshot iceberg table
+    // equals that in the original delta lake table
+    List<String> deltaTableDataFilePaths =
+        deltaLog.update().getAllFiles().stream()
+            .map(f -> getFullFilePath(f.getPath(), deltaLog.getPath().toString()))
+            .collect(Collectors.toList());
+    icebergTable
+        .currentSnapshot()
+        .addedDataFiles(icebergTable.io())
+        .forEach(
+            dataFile -> {
+              Assert.assertTrue(
+                  "The data file path should be the same as the original delta table",
+                  deltaTableDataFilePaths.contains(dataFile.path().toString()));
+            });
+  }
+
+  private Table getIcebergTable(String icebergTableIdentifier) {
+    CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog();
+    Spark3Util.CatalogAndIdentifier catalogAndIdent =
+        Spark3Util.catalogAndIdentifier(
+            "test catalog", spark, icebergTableIdentifier, defaultCatalog);
+    return Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name())
+        .loadTable(TableIdentifier.parse(catalogAndIdent.identifier().toString()));
+  }
+
+  private String destName(String catalogName, String dest) {
+    if (catalogName.equals(defaultSparkCatalog)) {
+      return NAMESPACE + "." + catalogName + "_" + dest;
+    }
+    return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest;
+  }
+
+  /**
+   * Add parquet files manually to a delta lake table to mock the situation that some data files are
+   * not in the same location as the delta lake table. The case that {@link AddFile#getPath()} or
+   * {@link RemoveFile#getPath()} returns absolute path.
+   *
+   * <p>The known <a href="https://github.com/delta-io/connectors/issues/380">issue</a> make it
+   * necessary to manually rebuild the AddFile to avoid deserialization error when committing the
+   * transaction.
+   */
+  private void addExternalDatafiles(
+      String targetDeltaTableLocation, String sourceDeltaTableLocation) {
+    DeltaLog targetLog =
+        DeltaLog.forTable(spark.sessionState().newHadoopConf(), targetDeltaTableLocation);
+    OptimisticTransaction transaction = targetLog.startTransaction();
+    DeltaLog sourceLog =
+        DeltaLog.forTable(spark.sessionState().newHadoopConf(), sourceDeltaTableLocation);
+    List<AddFile> newFiles =
+        sourceLog.update().getAllFiles().stream()
+            .map(
+                f ->
+                    AddFile.builder(
+                            getFullFilePath(f.getPath(), sourceLog.getPath().toString()),
+                            f.getPartitionValues(),
+                            f.getSize(),
+                            System.currentTimeMillis(),
+                            true)
+                        .build())
+            .collect(Collectors.toList());
+    try {
+      transaction.commit(newFiles, new Operation(Operation.Name.UPDATE), "Delta-Lake/2.2.0");
+    } catch (DeltaConcurrentModificationException e) {
+      // handle exception here

Review Comment:
   nit: I think the comment can be removed because it's obvious that the excepion is being handled here



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.delta;
+
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+
+/** An example class shows how to use the delta lake migration actions in SparkContext. */
+class DeltaLakeToIcebergMigrationSparkIntegration {
+
+  private DeltaLakeToIcebergMigrationSparkIntegration() {}
+
+  /**
+   * Example of creating a snapshot a delta table to iceberg table action in SparkContext.

Review Comment:
   I think this sentence is missing something



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+@SuppressWarnings("VisibilityModifier")
+public abstract class SparkDeltaLakeSnapshotTestBase {
+  protected static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore();
+    metastore.start();
+    SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf();
+
+    SparkDeltaLakeSnapshotTestBase.spark =
+        SparkSession.builder()
+            .master("local[2]")
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config(
+                "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname,
+                hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+            .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            // Needed for Delta Lake tests
+            .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    SparkDeltaLakeSnapshotTestBase.catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    SparkDeltaLakeSnapshotTestBase.catalog = null;
+    metastore.stop();
+    SparkDeltaLakeSnapshotTestBase.metastore = null;
+    spark.stop();
+    SparkDeltaLakeSnapshotTestBase.spark = null;
+  }
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  protected final String catalogName;

Review Comment:
   can we move all of those fields to the top? 



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,469 @@
+/*
+ * 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.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.Operation;
+import io.delta.standalone.OptimisticTransaction;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import io.delta.standalone.exceptions.DeltaConcurrentModificationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase {
+  private static final String row1 =
+      "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}},"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}";
+  private static final String row2 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},"
+          + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}";
+  private static final String row3 =
+      "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":"
+          + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}";
+  private static final String row4 =
+      "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}},"
+          + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":"
+          + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}";
+  private static final String row5 =
+      "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}],"
+          + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}},"
+          + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],"
+          + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}";
+  private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source";
+  private static final String DELTA_SOURCE_VALUE = "delta";
+  private static final String ORIGINAL_LOCATION_PROP = "original_location";
+  private static final String NAMESPACE = "default";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkSessionCatalog.class.getName(),
+        ImmutableMap.of(
+            "type",
+            "hive",
+            "default-namespace",
+            "default",
+            "parquet-enabled",
+            "true",
+            "cache-enabled",
+            "false" // Spark will delete tables using v1, leaving the cache out of sync
+            )
+      }
+    };
+  }
+
+  @Rule public TemporaryFolder temp1 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp2 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp3 = new TemporaryFolder();
+  @Rule public TemporaryFolder temp4 = new TemporaryFolder();
+
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = "external_data_files_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName());
+  }
+
+  /**
+   * The test hardcode a nested dataframe to test the snapshot feature. The schema of created
+   * dataframe is:
+   *
+   * <pre>
+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * </pre>
+   *
+   * The dataframe content is (by calling df.show()):
+   *
+   * <pre>
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * </pre>
+   */
+  @Before
+  public void before() {
+    try {
+      File partitionedFolder = temp1.newFolder();
+      File unpartitionedFolder = temp2.newFolder();
+      File newIcebergTableFolder = temp3.newFolder();
+      File externalDataFilesTableFolder = temp4.newFolder();
+      partitionedLocation = partitionedFolder.toURI().toString();
+      unpartitionedLocation = unpartitionedFolder.toURI().toString();
+      newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+      externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName);
+
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier));
+
+    // hard code the dataframe
+    List<String> jsonList = Lists.newArrayList();
+    jsonList.add(row1);
+    jsonList.add(row2);
+    jsonList.add(row3);
+    jsonList.add(row4);
+    jsonList.add(row5);
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    SQLContext sqlContext = new SQLContext(javaSparkContext);
+    JavaRDD<String> rdd = javaSparkContext.parallelize(jsonList);
+    Dataset<Row> df = sqlContext.read().json(rdd);
+
+    // write to delta tables
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .partitionBy("id")
+        .option("path", partitionedLocation)
+        .saveAsTable(partitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", unpartitionedLocation)
+        .saveAsTable(unpartitionedIdentifier);
+
+    df.write()
+        .format("delta")
+        .mode(SaveMode.Append)
+        .option("path", externalDataFilesTableLocation)
+        .saveAsTable(externalDataFilesIdentifier);
+
+    // Delete a record from the table
+    spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3");
+    spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3");
+
+    // Update a record
+    spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1");
+    spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1");
+  }
+
+  @After
+  public void after() {
+    // Drop delta lake tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName)));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, partitionedLocation);
+  }
+
+  @Test
+  public void testBasicSnapshotUnpartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotWithNewLocation() {
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, partitionedLocation)
+            .tableLocation(newIcebergTableLocation)
+            .execute();
+
+    checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation);
+  }
+
+  @Test
+  public void testSnapshotWithAdditionalProperties() {
+    // add some properties to the original delta table
+    spark.sql(
+        "ALTER TABLE "
+            + unpartitionedIdentifier
+            + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')");
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, unpartitionedLocation)
+            .tableProperty("test1", "test1")
+            .tableProperties(
+                ImmutableMap.of(
+                    "test2", "test2", "test3", "test3", "test4",
+                    "test4")) // add additional iceberg table properties
+            .execute();
+
+    checkSnapshotIntegrity(
+        unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation);
+    checkIcebergTableProperties(
+        newTableIdentifier,
+        ImmutableMap.of(
+            "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3",
+            "test4", "test4"),
+        unpartitionedLocation);
+  }
+
+  @Test
+  public void testSnapshotTableWithExternalDataFiles() {
+    // Add parquet files to default.external_data_files_table. The newly added parquet files
+    // are not at the same location as the table.
+    addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation);
+
+    String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files");
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation);
+  }
+
+  private void checkSnapshotIntegrity(
+      String deltaTableLocation,
+      String deltaTableIdentifier,
+      String icebergTableIdentifier,
+      SnapshotDeltaLakeTable.Result snapshotReport) {
+    DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation);
+
+    List<Row> deltaTableContents =
+        spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList();
+    List<Row> icebergTableContents =
+        spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList();
+
+    Assert.assertEquals(
+        "The original table and the transformed one should have the same size",
+        deltaTableContents.size(),
+        icebergTableContents.size());
+    Assert.assertTrue(

Review Comment:
   same for the other checks. Note that this will actually show the content of `icebergTableContents` if the assertion ever fails, so this is much better/easier in terms of debugging tests. This should also be done for all the other checks of that form



##########
delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.delta;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+@SuppressWarnings("VisibilityModifier")
+public abstract class SparkDeltaLakeSnapshotTestBase {
+  protected static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore();
+    metastore.start();
+    SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf();
+
+    SparkDeltaLakeSnapshotTestBase.spark =
+        SparkSession.builder()
+            .master("local[2]")
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config(
+                "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname,
+                hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))
+            .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            // Needed for Delta Lake tests
+            .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    SparkDeltaLakeSnapshotTestBase.catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    SparkDeltaLakeSnapshotTestBase.catalog = null;
+    metastore.stop();
+    SparkDeltaLakeSnapshotTestBase.metastore = null;
+    spark.stop();
+    SparkDeltaLakeSnapshotTestBase.spark = null;
+  }
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  protected final String catalogName;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table");

Review Comment:
   seems to be unused



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org