amogh-jahagirdar commented on code in PR #5331:
URL: https://github.com/apache/iceberg/pull/5331#discussion_r931828413


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java:
##########
@@ -293,7 +293,7 @@ private static PartitionSpec identitySpec(Schema schema, 
Collection<Column> colu
     return identitySpec(schema, names);
   }
 
-  private static PartitionSpec identitySpec(Schema schema, List<String> 
partitionNames) {
+  public static PartitionSpec identitySpec(Schema schema, List<String> 
partitionNames) {

Review Comment:
   Why is the change in access modifiers?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.actions.AddFile;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTypeToType;
+import org.apache.iceberg.spark.SparkTypeVisitor;
+import org.apache.iceberg.spark.source.StagedSparkTable;
+import org.apache.iceberg.types.Type;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.StagingTableCatalog;
+import org.apache.spark.sql.connector.expressions.LogicalExpressions;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Some;
+import scala.collection.JavaConverters;
+
+/**
+ * Takes a Delta Lake table and attempts to transform it into an Iceberg table 
in the same location with the same
+ * identifier. Once complete the identifier which previously referred to a 
non-Iceberg table will refer to the newly
+ * migrated Iceberg table.
+ */
+public class MigrateDeltaLakeTableSparkAction
+    implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class);
+  private static final String BACKUP_SUFFIX = "_BACKUP_";
+
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final SparkSession spark;
+  private final DeltaLog deltaLog;
+  private final StagingTableCatalog destCatalog;
+  private final String location;
+  private final Identifier identifier;
+  private final Identifier backupIdent;
+
+  MigrateDeltaLakeTableSparkAction(
+      SparkSession spark,
+      CatalogPlugin sourceCatalog,
+      Identifier sourceTableIdent
+  ) {
+    this.spark = spark;
+    this.destCatalog = checkDestinationCatalog(sourceCatalog);
+    this.identifier = sourceTableIdent;
+    this.backupIdent = Identifier.of(sourceTableIdent.namespace(), 
sourceTableIdent.name() + BACKUP_SUFFIX);
+    try {
+      CatalogTable tableMetadata = 
spark.sessionState().catalogManager().v1SessionCatalog()
+          .getTableMetadata(new TableIdentifier(sourceTableIdent.name()));
+      this.location = tableMetadata.location().getPath();
+      this.deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), 
tableMetadata.location().getPath());
+    } catch (NoSuchTableException | NoSuchDatabaseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Result execute() {
+    // Rename the table
+    renameAndBackupSourceTable();
+
+    // Get a DeltaLog instance and retrieve the partitions (if applicable) of 
the table
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    List<SparkTableUtil.SparkPartition> partitions =
+        getSparkPartitionsFromDeltaSnapshot(updatedSnapshot, 
deltaLog.getPath());
+
+    StructType structType = getStructTypeFromDeltaSnapshot(updatedSnapshot);
+
+    Table icebergTable;
+    StagedSparkTable stagedTable = null;
+    boolean error = true;
+    try {
+      stagedTable = stageDestTable(
+              updatedSnapshot, location, destCatalog, identifier, structType, 
additionalProperties);
+      icebergTable = stagedTable.table();
+
+      PartitionSpec partitionSpec = 
getPartitionSpecFromDeltaSnapshot(updatedSnapshot, structType);
+      String stagingLocation = 
SparkTableUtil.getIcebergMetadataLocation(icebergTable);
+
+      SparkTableUtil.importSparkTable(
+              spark,
+              new TableIdentifier(backupIdent.name(), 
Some.apply(backupIdent.namespace()[0])),
+              icebergTable,
+              stagingLocation,
+              Collections.emptyMap(),
+              false,
+              partitionSpec,
+              partitions
+      );
+
+      stagedTable.commitStagedChanges();
+      error = false;
+    } finally {
+      if (error) {
+        LOG.error("Failed to perform the migration, aborting table creation 
and restoring the original table");
+        restoreSourceTable(destCatalog, backupIdent, identifier);
+        if (stagedTable != null) {
+          try {
+            stagedTable.abortStagedChanges();
+          } catch (Exception abortException) {
+            LOG.error("Cannot abort staged changes", abortException);
+          }
+        }
+      }
+    }
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long numFilesMigrated = 
Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info("Successfully loaded Iceberg metadata for {} files to {}", 
numFilesMigrated, identifier);
+    return new BaseMigrateDeltaLakeTableActionResult(numFilesMigrated);
+  }
+
+  private static void restoreSourceTable(StagingTableCatalog 
destinationCatalog,
+                                  Identifier backupIdent, Identifier 
sourceTableIdent) {
+    try {
+      destinationCatalog.renameTable(backupIdent, sourceTableIdent);
+    } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) {
+      LOG.error("Cannot restore the original table, the backup table {} cannot 
be found", backupIdent, e);
+
+    } catch 
(org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) {
+      LOG.error("Cannot restore the original table, a table with the original 
name exists. " +
+              "Use the backup table {} to restore the original table 
manually.", backupIdent, e);
+    }
+  }
+
+  private static List<SparkTableUtil.SparkPartition> 
getSparkPartitionsFromDeltaSnapshot(
+      io.delta.standalone.Snapshot updatedSnapshot,
+      Path deltaLogPath
+  ) {
+    return updatedSnapshot.getAllFiles()
+        .stream()
+        // Map each partition to the list of files within it
+        .collect(Collectors.groupingBy(AddFile::getPartitionValues))
+        .entrySet()
+        .stream()
+        .map(entry -> {
+              // We don't care what value we take since they will all have the 
same prefix.
+              // The arbitrary file will have a path that looks like 
"partition1/partition2/file.parquet,
+              // We're interested in the part prior to the filename
+              AddFile addFile = entry.getValue().get(0);
+              String pathBeforeFileName = addFile.getPath().substring(0, 
addFile.getPath().lastIndexOf("/"));
+              String fullPath = new Path(deltaLogPath, 
pathBeforeFileName).toString();
+
+              return new SparkTableUtil.SparkPartition(
+                  entry.getKey(), // Map containing name and values of 
partitions
+                  fullPath,
+                  // Delta tables only support parquet
+                  "parquet"
+              );
+        }
+        )
+        .collect(Collectors.toList());
+  }
+
+  private static PartitionSpec getPartitionSpecFromDeltaSnapshot(

Review Comment:
   I get what the function is doing but does it need to be static?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.actions.AddFile;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTypeToType;
+import org.apache.iceberg.spark.SparkTypeVisitor;
+import org.apache.iceberg.spark.source.StagedSparkTable;
+import org.apache.iceberg.types.Type;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.StagingTableCatalog;
+import org.apache.spark.sql.connector.expressions.LogicalExpressions;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Some;
+import scala.collection.JavaConverters;
+
+/**
+ * Takes a Delta Lake table and attempts to transform it into an Iceberg table 
in the same location with the same
+ * identifier. Once complete the identifier which previously referred to a 
non-Iceberg table will refer to the newly
+ * migrated Iceberg table.
+ */
+public class MigrateDeltaLakeTableSparkAction
+    implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class);
+  private static final String BACKUP_SUFFIX = "_BACKUP_";
+
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final SparkSession spark;
+  private final DeltaLog deltaLog;
+  private final StagingTableCatalog destCatalog;
+  private final String location;
+  private final Identifier identifier;
+  private final Identifier backupIdent;
+
+  MigrateDeltaLakeTableSparkAction(
+      SparkSession spark,
+      CatalogPlugin sourceCatalog,
+      Identifier sourceTableIdent
+  ) {
+    this.spark = spark;
+    this.destCatalog = checkDestinationCatalog(sourceCatalog);
+    this.identifier = sourceTableIdent;
+    this.backupIdent = Identifier.of(sourceTableIdent.namespace(), 
sourceTableIdent.name() + BACKUP_SUFFIX);
+    try {
+      CatalogTable tableMetadata = 
spark.sessionState().catalogManager().v1SessionCatalog()
+          .getTableMetadata(new TableIdentifier(sourceTableIdent.name()));
+      this.location = tableMetadata.location().getPath();
+      this.deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), 
tableMetadata.location().getPath());
+    } catch (NoSuchTableException | NoSuchDatabaseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Result execute() {
+    // Rename the table
+    renameAndBackupSourceTable();
+
+    // Get a DeltaLog instance and retrieve the partitions (if applicable) of 
the table
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    List<SparkTableUtil.SparkPartition> partitions =
+        getSparkPartitionsFromDeltaSnapshot(updatedSnapshot, 
deltaLog.getPath());
+
+    StructType structType = getStructTypeFromDeltaSnapshot(updatedSnapshot);
+
+    Table icebergTable;
+    StagedSparkTable stagedTable = null;
+    boolean error = true;
+    try {
+      stagedTable = stageDestTable(
+              updatedSnapshot, location, destCatalog, identifier, structType, 
additionalProperties);
+      icebergTable = stagedTable.table();
+
+      PartitionSpec partitionSpec = 
getPartitionSpecFromDeltaSnapshot(updatedSnapshot, structType);
+      String stagingLocation = 
SparkTableUtil.getIcebergMetadataLocation(icebergTable);
+
+      SparkTableUtil.importSparkTable(
+              spark,
+              new TableIdentifier(backupIdent.name(), 
Some.apply(backupIdent.namespace()[0])),
+              icebergTable,
+              stagingLocation,
+              Collections.emptyMap(),
+              false,
+              partitionSpec,
+              partitions
+      );
+
+      stagedTable.commitStagedChanges();
+      error = false;
+    } finally {
+      if (error) {
+        LOG.error("Failed to perform the migration, aborting table creation 
and restoring the original table");
+        restoreSourceTable(destCatalog, backupIdent, identifier);
+        if (stagedTable != null) {
+          try {
+            stagedTable.abortStagedChanges();
+          } catch (Exception abortException) {
+            LOG.error("Cannot abort staged changes", abortException);
+          }
+        }
+      }
+    }
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long numFilesMigrated = 
Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info("Successfully loaded Iceberg metadata for {} files to {}", 
numFilesMigrated, identifier);
+    return new BaseMigrateDeltaLakeTableActionResult(numFilesMigrated);
+  }
+
+  private static void restoreSourceTable(StagingTableCatalog 
destinationCatalog,
+                                  Identifier backupIdent, Identifier 
sourceTableIdent) {
+    try {
+      destinationCatalog.renameTable(backupIdent, sourceTableIdent);
+    } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) {
+      LOG.error("Cannot restore the original table, the backup table {} cannot 
be found", backupIdent, e);
+
+    } catch 
(org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) {
+      LOG.error("Cannot restore the original table, a table with the original 
name exists. " +
+              "Use the backup table {} to restore the original table 
manually.", backupIdent, e);
+    }
+  }
+
+  private static List<SparkTableUtil.SparkPartition> 
getSparkPartitionsFromDeltaSnapshot(

Review Comment:
   same, does it need to be static?



##########
core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+public class BaseMigrateDeltaLakeTableActionResult implements 
MigrateDeltaLakeTable.Result {
+
+  private final long numFilesImported;
+
+  public BaseMigrateDeltaLakeTableActionResult(long numFilesImported) {

Review Comment:
   Would it make sense to capture aspects like the total data size and how long 
the migration procedure took in the result?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.actions.AddFile;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTypeToType;
+import org.apache.iceberg.spark.SparkTypeVisitor;
+import org.apache.iceberg.spark.source.StagedSparkTable;
+import org.apache.iceberg.types.Type;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.StagingTableCatalog;
+import org.apache.spark.sql.connector.expressions.LogicalExpressions;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Some;
+import scala.collection.JavaConverters;
+
+/**
+ * Takes a Delta Lake table and attempts to transform it into an Iceberg table 
in the same location with the same
+ * identifier. Once complete the identifier which previously referred to a 
non-Iceberg table will refer to the newly
+ * migrated Iceberg table.
+ */
+public class MigrateDeltaLakeTableSparkAction
+    implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class);
+  private static final String BACKUP_SUFFIX = "_BACKUP_";
+
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final SparkSession spark;
+  private final DeltaLog deltaLog;
+  private final StagingTableCatalog destCatalog;
+  private final String location;
+  private final Identifier identifier;
+  private final Identifier backupIdent;
+
+  MigrateDeltaLakeTableSparkAction(
+      SparkSession spark,
+      CatalogPlugin sourceCatalog,
+      Identifier sourceTableIdent
+  ) {
+    this.spark = spark;
+    this.destCatalog = checkDestinationCatalog(sourceCatalog);
+    this.identifier = sourceTableIdent;
+    this.backupIdent = Identifier.of(sourceTableIdent.namespace(), 
sourceTableIdent.name() + BACKUP_SUFFIX);
+    try {
+      CatalogTable tableMetadata = 
spark.sessionState().catalogManager().v1SessionCatalog()
+          .getTableMetadata(new TableIdentifier(sourceTableIdent.name()));
+      this.location = tableMetadata.location().getPath();
+      this.deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), 
tableMetadata.location().getPath());
+    } catch (NoSuchTableException | NoSuchDatabaseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Result execute() {
+    // Rename the table
+    renameAndBackupSourceTable();
+
+    // Get a DeltaLog instance and retrieve the partitions (if applicable) of 
the table
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    List<SparkTableUtil.SparkPartition> partitions =
+        getSparkPartitionsFromDeltaSnapshot(updatedSnapshot, 
deltaLog.getPath());
+
+    StructType structType = getStructTypeFromDeltaSnapshot(updatedSnapshot);
+
+    Table icebergTable;
+    StagedSparkTable stagedTable = null;
+    boolean error = true;
+    try {
+      stagedTable = stageDestTable(
+              updatedSnapshot, location, destCatalog, identifier, structType, 
additionalProperties);
+      icebergTable = stagedTable.table();
+
+      PartitionSpec partitionSpec = 
getPartitionSpecFromDeltaSnapshot(updatedSnapshot, structType);
+      String stagingLocation = 
SparkTableUtil.getIcebergMetadataLocation(icebergTable);
+
+      SparkTableUtil.importSparkTable(
+              spark,
+              new TableIdentifier(backupIdent.name(), 
Some.apply(backupIdent.namespace()[0])),
+              icebergTable,
+              stagingLocation,
+              Collections.emptyMap(),
+              false,
+              partitionSpec,
+              partitions
+      );
+
+      stagedTable.commitStagedChanges();
+      error = false;
+    } finally {
+      if (error) {
+        LOG.error("Failed to perform the migration, aborting table creation 
and restoring the original table");
+        restoreSourceTable(destCatalog, backupIdent, identifier);
+        if (stagedTable != null) {
+          try {
+            stagedTable.abortStagedChanges();
+          } catch (Exception abortException) {
+            LOG.error("Cannot abort staged changes", abortException);
+          }
+        }
+      }
+    }
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long numFilesMigrated = 
Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info("Successfully loaded Iceberg metadata for {} files to {}", 
numFilesMigrated, identifier);
+    return new BaseMigrateDeltaLakeTableActionResult(numFilesMigrated);
+  }
+
+  private static void restoreSourceTable(StagingTableCatalog 
destinationCatalog,
+                                  Identifier backupIdent, Identifier 
sourceTableIdent) {
+    try {
+      destinationCatalog.renameTable(backupIdent, sourceTableIdent);
+    } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) {
+      LOG.error("Cannot restore the original table, the backup table {} cannot 
be found", backupIdent, e);
+
+    } catch 
(org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) {
+      LOG.error("Cannot restore the original table, a table with the original 
name exists. " +
+              "Use the backup table {} to restore the original table 
manually.", backupIdent, e);
+    }
+  }
+
+  private static List<SparkTableUtil.SparkPartition> 
getSparkPartitionsFromDeltaSnapshot(
+      io.delta.standalone.Snapshot updatedSnapshot,
+      Path deltaLogPath
+  ) {
+    return updatedSnapshot.getAllFiles()
+        .stream()
+        // Map each partition to the list of files within it
+        .collect(Collectors.groupingBy(AddFile::getPartitionValues))
+        .entrySet()
+        .stream()
+        .map(entry -> {
+              // We don't care what value we take since they will all have the 
same prefix.
+              // The arbitrary file will have a path that looks like 
"partition1/partition2/file.parquet,
+              // We're interested in the part prior to the filename
+              AddFile addFile = entry.getValue().get(0);
+              String pathBeforeFileName = addFile.getPath().substring(0, 
addFile.getPath().lastIndexOf("/"));
+              String fullPath = new Path(deltaLogPath, 
pathBeforeFileName).toString();
+
+              return new SparkTableUtil.SparkPartition(
+                  entry.getKey(), // Map containing name and values of 
partitions
+                  fullPath,
+                  // Delta tables only support parquet
+                  "parquet"
+              );
+        }
+        )
+        .collect(Collectors.toList());
+  }

Review Comment:
   Should we parallelize this computation? For example, concurrently across 
files update a grouping based on partition values. Could be overkill (depends 
on the number of files in the snapshot)



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java:
##########
@@ -380,7 +381,9 @@ private static Iterator<ManifestFile> 
buildManifest(SerializableConfiguration co
    */
   public static void importSparkTable(SparkSession spark, TableIdentifier 
sourceTableIdent, Table targetTable,
                                       String stagingDir, Map<String, String> 
partitionFilter,
-                                      boolean checkDuplicateFiles) {
+                                      boolean checkDuplicateFiles,
+                                      PartitionSpec nullableSpec,
+                                      List<SparkPartition> nullablePartitions) 
{

Review Comment:
   I get we need to pass in the computed partition spec and spark partitions, 
but don't think we should change any public API signatures, we can add a new 
one.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to