JonasJ-ap commented on code in PR #6449:
URL: https://github.com/apache/iceberg/pull/6449#discussion_r1084798607


##########
delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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 static org.apache.spark.sql.functions.current_date;
+import static org.apache.spark.sql.functions.date_add;
+import static org.apache.spark.sql.functions.expr;
+
+import 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.commons.codec.DecoderException;
+import org.apache.commons.codec.net.URLCodec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.delta.catalog.DeltaCatalog;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+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 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 = "delta_conversion_test";
+  private static final String defaultSparkCatalog = "spark_catalog";
+  private static final String icebergCatalogName = "iceberg_hive";
+  private String partitionedIdentifier;
+  private String unpartitionedIdentifier;
+  private String externalDataFilesIdentifier;
+  private String typeTestIdentifier;
+  private final String partitionedTableName = "partitioned_table";
+  private final String unpartitionedTableName = "unpartitioned_table";
+  private final String externalDataFilesTableName = 
"external_data_files_table";
+  private final String typeTestTableName = "type_test_table";
+  private final String snapshotPartitionedTableName = 
"iceberg_partitioned_table";
+  private final String snapshotUnpartitionedTableName = 
"iceberg_unpartitioned_table";
+  private final String snapshotExternalDataFilesTableName = 
"iceberg_external_data_files_table";
+  private final String snapshotNewTableLocationTableName = 
"iceberg_new_table_location_table";
+  private final String snapshotAdditionalPropertiesTableName =
+      "iceberg_additional_properties_table";
+  private final String snapshotTypeTestTableName = "iceberg_type_test_table";
+  private String partitionedLocation;
+  private String unpartitionedLocation;
+  private String newIcebergTableLocation;
+  private String externalDataFilesTableLocation;
+  private String typeTestTableLocation;
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      new Object[] {
+        icebergCatalogName,
+        SparkCatalog.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();
+  @Rule public TemporaryFolder temp5 = new TemporaryFolder();
+
+  public TestSnapshotDeltaLakeTable(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, 
DeltaCatalog.class.getName());
+  }
+
+  @Before
+  public void before() throws IOException {
+    File partitionedFolder = temp1.newFolder();
+    File unpartitionedFolder = temp2.newFolder();
+    File newIcebergTableFolder = temp3.newFolder();
+    File externalDataFilesTableFolder = temp4.newFolder();
+    File typeTestTableFolder = temp5.newFolder();
+    partitionedLocation = partitionedFolder.toURI().toString();
+    unpartitionedLocation = unpartitionedFolder.toURI().toString();
+    newIcebergTableLocation = newIcebergTableFolder.toURI().toString();
+    externalDataFilesTableLocation = 
externalDataFilesTableFolder.toURI().toString();
+    typeTestTableLocation = typeTestTableFolder.toURI().toString();
+
+    spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", NAMESPACE));
+
+    partitionedIdentifier = destName(defaultSparkCatalog, 
partitionedTableName);
+    unpartitionedIdentifier = destName(defaultSparkCatalog, 
unpartitionedTableName);
+    externalDataFilesIdentifier = destName(defaultSparkCatalog, 
externalDataFilesTableName);
+    typeTestIdentifier = destName(defaultSparkCatalog, typeTestTableName);
+
+    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));
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", typeTestIdentifier));
+
+    // generate the dataframe
+    Dataset<Row> nestedDataFrame = nestedDataFrame();
+    Dataset<Row> typeTestDataFrame = typeTestDataFrame();
+
+    // write to delta tables
+    writeDeltaTable(nestedDataFrame, partitionedIdentifier, 
partitionedLocation, "id");
+    writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, 
unpartitionedLocation, null);
+    writeDeltaTable(
+        nestedDataFrame, externalDataFilesIdentifier, 
externalDataFilesTableLocation, null);
+    writeDeltaTable(typeTestDataFrame, typeTestIdentifier, 
typeTestTableLocation, "stringCol");
+
+    // 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)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, 
externalDataFilesTableName)));
+    spark.sql(
+        String.format("DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, 
typeTestTableName)));
+
+    // Drop iceberg tables.
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(icebergCatalogName, 
snapshotPartitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s",
+            destName(icebergCatalogName, snapshotUnpartitionedTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s",
+            destName(icebergCatalogName, snapshotExternalDataFilesTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s",
+            destName(icebergCatalogName, snapshotNewTableLocationTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s",
+            destName(icebergCatalogName, 
snapshotAdditionalPropertiesTableName)));
+    spark.sql(
+        String.format(
+            "DROP TABLE IF EXISTS %s", destName(icebergCatalogName, 
snapshotTypeTestTableName)));
+
+    spark.sql(String.format("DROP DATABASE IF EXISTS %s", NAMESPACE));
+  }
+
+  @Test
+  public void testBasicSnapshotPartitioned() {
+    String newTableIdentifier = destName(icebergCatalogName, 
snapshotPartitionedTableName);
+    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, 
snapshotUnpartitionedTableName);
+    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, 
snapshotNewTableLocationTableName);
+    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, 
snapshotAdditionalPropertiesTableName);
+    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, 
snapshotExternalDataFilesTableName);
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, externalDataFilesTableLocation)
+            .execute();
+    checkSnapshotIntegrity(
+        externalDataFilesTableLocation, externalDataFilesIdentifier, 
newTableIdentifier, result);
+    checkIcebergTableLocation(newTableIdentifier, 
externalDataFilesTableLocation);
+    checkDataFilePathsIntegrity(newTableIdentifier, 
externalDataFilesTableLocation);
+  }
+
+  @Test
+  public void testSnapshotSupportedTypes() {
+    String newTableIdentifier = destName(icebergCatalogName, 
snapshotTypeTestTableName);
+    SnapshotDeltaLakeTable.Result result =
+        DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable(
+                spark, newTableIdentifier, typeTestTableLocation)
+            .tableProperty(TableProperties.PARQUET_VECTORIZATION_ENABLED, 
"false")

Review Comment:
   The `testTypeDataFrame` includes a timestamp column which is default to 
INT96 by delta lake/spark. 
   According to #4196 and #4200 , currently the vectorized reader does not 
support `INT96`, so we must disable it.
   
   I will also add this to the doc #6600 



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to