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 2022/12/21 19:21:52 UTC

[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6449: WIP: Core, Spark: Adding support for Migrating Delta Lake Table to Iceberg Table

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


##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";

Review Comment:
   nit: you mean suffix?



##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";
+  private final String avroPostfix = ".avro";
+  private final String orcPostfix = ".orc";
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final DeltaLog deltaLog;
+  private final Catalog icebergCatalog;
+  private final String deltaTableLocation;
+  private final TableIdentifier newTableIdentifier;
+  private final Configuration hadoopConfiguration;
+
+  public BaseMigrateDeltaLakeTableAction(
+      Catalog icebergCatalog,
+      String deltaTableLocation,
+      TableIdentifier newTableIdentifier,
+      Configuration hadoopConfiguration) {
+    this.icebergCatalog = icebergCatalog;
+    this.deltaTableLocation = deltaTableLocation;
+    this.newTableIdentifier = newTableIdentifier;
+    this.hadoopConfiguration = hadoopConfiguration;
+    // TODO: check whether we can retrieve hadoopConfiguration directly
+    this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation);
+  }
+
+  @Override
+  public MigrateDeltaLakeTable tableProperties(Map<String, String> properties) {
+    additionalProperties.putAll(properties);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema());

Review Comment:
   nit: `convertDeltaLakeSchemaToSchema` sounds quite convoluted, what about just `convertDeltaLakeSchema`?



##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";
+  private final String avroPostfix = ".avro";
+  private final String orcPostfix = ".orc";
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final DeltaLog deltaLog;
+  private final Catalog icebergCatalog;
+  private final String deltaTableLocation;
+  private final TableIdentifier newTableIdentifier;
+  private final Configuration hadoopConfiguration;
+
+  public BaseMigrateDeltaLakeTableAction(
+      Catalog icebergCatalog,
+      String deltaTableLocation,
+      TableIdentifier newTableIdentifier,
+      Configuration hadoopConfiguration) {
+    this.icebergCatalog = icebergCatalog;
+    this.deltaTableLocation = deltaTableLocation;
+    this.newTableIdentifier = newTableIdentifier;
+    this.hadoopConfiguration = hadoopConfiguration;
+    // TODO: check whether we can retrieve hadoopConfiguration directly
+    this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation);
+  }
+
+  @Override
+  public MigrateDeltaLakeTable tableProperties(Map<String, String> properties) {
+    additionalProperties.putAll(properties);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema());
+    PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema);
+    // TODO: check whether we need more info when initializing the table
+    Table icebergTable =
+        this.icebergCatalog.createTable(
+            newTableIdentifier,
+            schema,
+            partitionSpec,
+            destTableProperties(
+                updatedSnapshot, this.deltaTableLocation, this.additionalProperties));
+
+    copyFromDeltaLakeToIceberg(icebergTable, partitionSpec);
+
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long totalDataFiles =
+        Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info(
+        "Successfully loaded Iceberg metadata for {} files to {}",
+        totalDataFiles,
+        deltaTableLocation);
+    return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles);
+  }
+
+  /** TODO: check the correctness for nested schema */
+  private Schema convertDeltaLakeSchemaToSchema(io.delta.standalone.types.StructType deltaSchema) {
+    Type converted =
+        DeltaLakeDataTypeVisitor.visit(deltaSchema, new DeltaLakeTypeToType(deltaSchema));
+    return new Schema(converted.asNestedType().asStructType().fields());
+  }
+
+  private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) {
+    List<String> partitionNames = deltaLog.snapshot().getMetadata().getPartitionColumns();
+    if (partitionNames.isEmpty()) {
+      return PartitionSpec.unpartitioned();
+    }
+
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+    for (String partitionName : partitionNames) {
+      builder.identity(partitionName);
+    }
+    return builder.build();
+  }
+
+  private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) {
+    // TODO: double check the arguments' meaning here
+    Iterator<VersionLog> it = deltaLog.getChanges(0, false);

Review Comment:
   nit: we can comment about the meaning using `deltaLog.getChanges(/* some meaning */ 0, /* some meaning */ false);`
   
   or 
   
   ```
   deltaLog.getChanges(
     0, // some meaning
     false); // some meaning



##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";
+  private final String avroPostfix = ".avro";
+  private final String orcPostfix = ".orc";
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final DeltaLog deltaLog;
+  private final Catalog icebergCatalog;
+  private final String deltaTableLocation;
+  private final TableIdentifier newTableIdentifier;
+  private final Configuration hadoopConfiguration;
+
+  public BaseMigrateDeltaLakeTableAction(
+      Catalog icebergCatalog,
+      String deltaTableLocation,
+      TableIdentifier newTableIdentifier,
+      Configuration hadoopConfiguration) {
+    this.icebergCatalog = icebergCatalog;
+    this.deltaTableLocation = deltaTableLocation;
+    this.newTableIdentifier = newTableIdentifier;
+    this.hadoopConfiguration = hadoopConfiguration;
+    // TODO: check whether we can retrieve hadoopConfiguration directly
+    this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation);
+  }
+
+  @Override
+  public MigrateDeltaLakeTable tableProperties(Map<String, String> properties) {
+    additionalProperties.putAll(properties);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema());
+    PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema);
+    // TODO: check whether we need more info when initializing the table
+    Table icebergTable =
+        this.icebergCatalog.createTable(
+            newTableIdentifier,
+            schema,
+            partitionSpec,
+            destTableProperties(
+                updatedSnapshot, this.deltaTableLocation, this.additionalProperties));
+
+    copyFromDeltaLakeToIceberg(icebergTable, partitionSpec);
+
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long totalDataFiles =
+        Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info(
+        "Successfully loaded Iceberg metadata for {} files to {}",
+        totalDataFiles,
+        deltaTableLocation);
+    return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles);
+  }
+
+  /** TODO: check the correctness for nested schema */
+  private Schema convertDeltaLakeSchemaToSchema(io.delta.standalone.types.StructType deltaSchema) {
+    Type converted =
+        DeltaLakeDataTypeVisitor.visit(deltaSchema, new DeltaLakeTypeToType(deltaSchema));
+    return new Schema(converted.asNestedType().asStructType().fields());
+  }
+
+  private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) {
+    List<String> partitionNames = deltaLog.snapshot().getMetadata().getPartitionColumns();
+    if (partitionNames.isEmpty()) {
+      return PartitionSpec.unpartitioned();
+    }
+
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+    for (String partitionName : partitionNames) {
+      builder.identity(partitionName);
+    }
+    return builder.build();
+  }
+
+  private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) {
+    // TODO: double check the arguments' meaning here
+    Iterator<VersionLog> it = deltaLog.getChanges(0, false);
+
+    while (it.hasNext()) {
+      VersionLog versionLog = it.next();
+      List<Action> actions = versionLog.getActions();
+
+      // We first need to iterate through to see what kind of transaction this was. There are 3
+      // cases:
+      // 1. AppendFile - when there are only AddFile instances (an INSERT on the table)
+      // 2. DeleteFiles - when there are only RemoveFile instances (a DELETE where all the records
+      // of file(s) were removed
+      // 3. OverwriteFiles - when there are a mix of AddFile and RemoveFile (a DELETE/UPDATE)
+
+      // Create a map of Delta Lake Action (AddFile, RemoveFile, etc.) --> List<Action>
+      Map<String, List<Action>> deltaLakeActionMap =
+          actions.stream()
+              .filter(action -> action instanceof AddFile || action instanceof RemoveFile)
+              .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName()));
+
+      // Scan the map so that we know what type of transaction this will be in Iceberg
+      IcebergTransactionType icebergTransactionType =
+          getIcebergTransactionTypeFromDeltaActions(deltaLakeActionMap);
+      if (icebergTransactionType == null) {
+        // TODO: my understanding here is that if the transaction type is undefined,
+        //  we can no longer continue even the next versionLog contains valid transaction type
+        //  may need further check
+        return;
+      }
+
+      List<DataFile> filesToAdd = Lists.newArrayList();
+      List<DataFile> filesToRemove = Lists.newArrayList();
+      for (Action action : Iterables.concat(deltaLakeActionMap.values())) {
+        DataFile dataFile = buildDataFileFromAction(action, table, spec);
+        if (action instanceof AddFile) {
+          filesToAdd.add(dataFile);
+        } else if (action instanceof RemoveFile) {
+          filesToRemove.add(dataFile);
+        } else {
+          // TODO: refactor this exception
+          throw new RuntimeException("Wrong action type");
+        }
+      }
+
+      switch (icebergTransactionType) {
+        case APPEND_FILES:
+          AppendFiles appendFiles = table.newAppend();
+          filesToAdd.forEach(appendFiles::appendFile);
+          appendFiles.commit();
+          break;
+        case DELETE_FILES:
+          DeleteFiles deleteFiles = table.newDelete();
+          filesToRemove.forEach(deleteFiles::deleteFile);
+          deleteFiles.commit();
+          break;
+        case OVERWRITE_FILES:
+          OverwriteFiles overwriteFiles = table.newOverwrite();
+          filesToAdd.forEach(overwriteFiles::addFile);
+          filesToRemove.forEach(overwriteFiles::deleteFile);
+          overwriteFiles.commit();
+          break;
+      }
+    }
+  }
+
+  @Nullable
+  private IcebergTransactionType getIcebergTransactionTypeFromDeltaActions(
+      Map<String, List<Action>> actionsMap) {
+    IcebergTransactionType icebergTransactionType;
+    if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && !actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.APPEND_FILES;
+    } else if (actionsMap.containsKey(RemoveFile.class.getSimpleName())
+        && !actionsMap.containsKey(AddFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.DELETE_FILES;
+    } else if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.OVERWRITE_FILES;
+    } else {
+      // Some other type of transaction, we can ignore
+      return null;
+    }
+    return icebergTransactionType;
+  }
+
+  private DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) {
+    String path;
+    long size;
+    Map<String, String> partitionValues;
+
+    if (action instanceof AddFile) {
+      AddFile addFile = (AddFile) action;
+      path = addFile.getPath();
+      size = addFile.getSize();
+      partitionValues = addFile.getPartitionValues();
+    } else if (action instanceof RemoveFile) {
+      RemoveFile removeFile = (RemoveFile) action;
+      path = removeFile.getPath();
+      size =
+          removeFile
+              .getSize()
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format("File %s removed with specifying a size", path)));
+      partitionValues =
+          Optional.ofNullable(removeFile.getPartitionValues())
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format(
+                              "File %s removed without specifying partition values", path)));
+    } else {
+      throw new IllegalStateException(
+          String.format(
+              "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName()));
+    }
+
+    String fullFilePath = deltaLog.getPath().toString() + File.separator + path;
+    FileFormat format = determineFileFormatFromPath(fullFilePath);
+
+    Metrics metrics = getMetricsForFile(table, fullFilePath, format);
+    String partition =
+        spec.fields().stream()
+            .map(PartitionField::name)
+            .map(name -> String.format("%s=%s", name, partitionValues.get(name)))
+            .collect(Collectors.joining("/"));
+
+    return DataFiles.builder(spec)
+        .withPath(fullFilePath)
+        .withFormat(format)
+        .withFileSizeInBytes(size)
+        .withMetrics(metrics)
+        .withPartitionPath(partition)
+        .withRecordCount(metrics.recordCount())
+        .build();
+  }
+
+  protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format);
+
+  private FileFormat determineFileFormatFromPath(String path) {
+    if (path.endsWith(parquetPostfix)) {
+      return FileFormat.PARQUET;
+    } else if (path.endsWith(avroPostfix)) {
+      return FileFormat.AVRO;
+    } else if (path.endsWith(orcPostfix)) {
+      return FileFormat.ORC;
+    } else {
+      throw new RuntimeException("The format of the file is unsupported: " + path);
+    }
+  }
+
+  private static Map<String, String> destTableProperties(
+      io.delta.standalone.Snapshot deltaSnapshot,
+      String tableLocation,
+      Map<String, String> additionalProperties) {
+    Map<String, String> properties = Maps.newHashMap();
+
+    properties.putAll(deltaSnapshot.getMetadata().getConfiguration());
+    properties.putAll(
+        ImmutableMap.of(
+            "provider",
+            "iceberg",
+            "migrated",

Review Comment:
   this is a bit generic, can we use something more indicative like `migration_source=delta`?



##########
core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import io.delta.standalone.types.ArrayType;
+import io.delta.standalone.types.DataType;
+import io.delta.standalone.types.MapType;
+import io.delta.standalone.types.StructField;
+import io.delta.standalone.types.StructType;
+import java.util.List;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class DeltaLakeDataTypeVisitor<T> {
+  public static <T> T visit(DataType type, DeltaLakeDataTypeVisitor<T> visitor) {
+    if (type instanceof StructType) {
+      StructField[] fields = ((StructType) type).getFields();
+      List<T> fieldResults = Lists.newArrayListWithExpectedSize(fields.length);
+
+      for (StructField field : fields) {
+        fieldResults.add(visitor.field(field, visit(field.getDataType(), visitor)));
+      }
+
+      return visitor.struct((StructType) type, fieldResults);
+
+    } else if (type instanceof MapType) {
+      return visitor.map(
+          (MapType) type,
+          visit(((MapType) type).getKeyType(), visitor),
+          visit(((MapType) type).getValueType(), visitor));
+
+    } else if (type instanceof ArrayType) {
+      return visitor.array((ArrayType) type, visit(((ArrayType) type).getElementType(), visitor));
+
+    } else {
+      return visitor.atomic(type);
+    }
+  }
+
+  public T struct(StructType struct, List<T> fieldResults) {
+    return null;

Review Comment:
   these need to be implemented



##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";
+  private final String avroPostfix = ".avro";
+  private final String orcPostfix = ".orc";
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final DeltaLog deltaLog;
+  private final Catalog icebergCatalog;
+  private final String deltaTableLocation;
+  private final TableIdentifier newTableIdentifier;
+  private final Configuration hadoopConfiguration;
+
+  public BaseMigrateDeltaLakeTableAction(
+      Catalog icebergCatalog,
+      String deltaTableLocation,
+      TableIdentifier newTableIdentifier,
+      Configuration hadoopConfiguration) {
+    this.icebergCatalog = icebergCatalog;
+    this.deltaTableLocation = deltaTableLocation;
+    this.newTableIdentifier = newTableIdentifier;
+    this.hadoopConfiguration = hadoopConfiguration;
+    // TODO: check whether we can retrieve hadoopConfiguration directly
+    this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation);
+  }
+
+  @Override
+  public MigrateDeltaLakeTable tableProperties(Map<String, String> properties) {
+    additionalProperties.putAll(properties);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema());
+    PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema);
+    // TODO: check whether we need more info when initializing the table
+    Table icebergTable =
+        this.icebergCatalog.createTable(
+            newTableIdentifier,
+            schema,
+            partitionSpec,
+            destTableProperties(
+                updatedSnapshot, this.deltaTableLocation, this.additionalProperties));
+
+    copyFromDeltaLakeToIceberg(icebergTable, partitionSpec);
+
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long totalDataFiles =
+        Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info(
+        "Successfully loaded Iceberg metadata for {} files to {}",
+        totalDataFiles,
+        deltaTableLocation);
+    return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles);
+  }
+
+  /** TODO: check the correctness for nested schema */
+  private Schema convertDeltaLakeSchemaToSchema(io.delta.standalone.types.StructType deltaSchema) {
+    Type converted =
+        DeltaLakeDataTypeVisitor.visit(deltaSchema, new DeltaLakeTypeToType(deltaSchema));
+    return new Schema(converted.asNestedType().asStructType().fields());
+  }
+
+  private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) {
+    List<String> partitionNames = deltaLog.snapshot().getMetadata().getPartitionColumns();
+    if (partitionNames.isEmpty()) {
+      return PartitionSpec.unpartitioned();
+    }
+
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+    for (String partitionName : partitionNames) {
+      builder.identity(partitionName);
+    }
+    return builder.build();
+  }
+
+  private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) {
+    // TODO: double check the arguments' meaning here
+    Iterator<VersionLog> it = deltaLog.getChanges(0, false);
+
+    while (it.hasNext()) {
+      VersionLog versionLog = it.next();
+      List<Action> actions = versionLog.getActions();
+
+      // We first need to iterate through to see what kind of transaction this was. There are 3
+      // cases:
+      // 1. AppendFile - when there are only AddFile instances (an INSERT on the table)
+      // 2. DeleteFiles - when there are only RemoveFile instances (a DELETE where all the records
+      // of file(s) were removed
+      // 3. OverwriteFiles - when there are a mix of AddFile and RemoveFile (a DELETE/UPDATE)
+
+      // Create a map of Delta Lake Action (AddFile, RemoveFile, etc.) --> List<Action>
+      Map<String, List<Action>> deltaLakeActionMap =
+          actions.stream()
+              .filter(action -> action instanceof AddFile || action instanceof RemoveFile)
+              .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName()));
+
+      // Scan the map so that we know what type of transaction this will be in Iceberg
+      IcebergTransactionType icebergTransactionType =
+          getIcebergTransactionTypeFromDeltaActions(deltaLakeActionMap);
+      if (icebergTransactionType == null) {
+        // TODO: my understanding here is that if the transaction type is undefined,
+        //  we can no longer continue even the next versionLog contains valid transaction type
+        //  may need further check
+        return;
+      }
+
+      List<DataFile> filesToAdd = Lists.newArrayList();
+      List<DataFile> filesToRemove = Lists.newArrayList();
+      for (Action action : Iterables.concat(deltaLakeActionMap.values())) {
+        DataFile dataFile = buildDataFileFromAction(action, table, spec);
+        if (action instanceof AddFile) {
+          filesToAdd.add(dataFile);
+        } else if (action instanceof RemoveFile) {
+          filesToRemove.add(dataFile);
+        } else {
+          // TODO: refactor this exception
+          throw new RuntimeException("Wrong action type");
+        }
+      }
+
+      switch (icebergTransactionType) {
+        case APPEND_FILES:
+          AppendFiles appendFiles = table.newAppend();
+          filesToAdd.forEach(appendFiles::appendFile);
+          appendFiles.commit();
+          break;
+        case DELETE_FILES:
+          DeleteFiles deleteFiles = table.newDelete();
+          filesToRemove.forEach(deleteFiles::deleteFile);
+          deleteFiles.commit();
+          break;
+        case OVERWRITE_FILES:
+          OverwriteFiles overwriteFiles = table.newOverwrite();
+          filesToAdd.forEach(overwriteFiles::addFile);
+          filesToRemove.forEach(overwriteFiles::deleteFile);
+          overwriteFiles.commit();
+          break;
+      }
+    }
+  }
+
+  @Nullable
+  private IcebergTransactionType getIcebergTransactionTypeFromDeltaActions(
+      Map<String, List<Action>> actionsMap) {
+    IcebergTransactionType icebergTransactionType;
+    if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && !actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.APPEND_FILES;
+    } else if (actionsMap.containsKey(RemoveFile.class.getSimpleName())
+        && !actionsMap.containsKey(AddFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.DELETE_FILES;
+    } else if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.OVERWRITE_FILES;
+    } else {
+      // Some other type of transaction, we can ignore
+      return null;
+    }
+    return icebergTransactionType;
+  }
+
+  private DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) {
+    String path;
+    long size;
+    Map<String, String> partitionValues;
+
+    if (action instanceof AddFile) {
+      AddFile addFile = (AddFile) action;
+      path = addFile.getPath();
+      size = addFile.getSize();
+      partitionValues = addFile.getPartitionValues();
+    } else if (action instanceof RemoveFile) {
+      RemoveFile removeFile = (RemoveFile) action;
+      path = removeFile.getPath();
+      size =
+          removeFile
+              .getSize()
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format("File %s removed with specifying a size", path)));
+      partitionValues =
+          Optional.ofNullable(removeFile.getPartitionValues())
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format(
+                              "File %s removed without specifying partition values", path)));
+    } else {
+      throw new IllegalStateException(
+          String.format(
+              "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName()));
+    }
+
+    String fullFilePath = deltaLog.getPath().toString() + File.separator + path;
+    FileFormat format = determineFileFormatFromPath(fullFilePath);
+
+    Metrics metrics = getMetricsForFile(table, fullFilePath, format);
+    String partition =
+        spec.fields().stream()
+            .map(PartitionField::name)
+            .map(name -> String.format("%s=%s", name, partitionValues.get(name)))
+            .collect(Collectors.joining("/"));
+
+    return DataFiles.builder(spec)
+        .withPath(fullFilePath)
+        .withFormat(format)
+        .withFileSizeInBytes(size)
+        .withMetrics(metrics)
+        .withPartitionPath(partition)
+        .withRecordCount(metrics.recordCount())
+        .build();
+  }
+
+  protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format);

Review Comment:
   agree, I was proposing `iceberg-deltalake`, because for example if we can also later have `iceberg-hudi`, and people can just take 1 dependency for their migration purpose, instead of multiple.



##########
core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.VersionLog;
+import io.delta.standalone.actions.Action;
+import io.delta.standalone.actions.AddFile;
+import io.delta.standalone.actions.RemoveFile;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.catalog.Catalog;
+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.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private final String parquetPostfix = ".parquet";
+  private final String avroPostfix = ".avro";
+  private final String orcPostfix = ".orc";
+  private final Map<String, String> additionalProperties = Maps.newHashMap();
+  private final DeltaLog deltaLog;
+  private final Catalog icebergCatalog;
+  private final String deltaTableLocation;
+  private final TableIdentifier newTableIdentifier;
+  private final Configuration hadoopConfiguration;
+
+  public BaseMigrateDeltaLakeTableAction(
+      Catalog icebergCatalog,
+      String deltaTableLocation,
+      TableIdentifier newTableIdentifier,
+      Configuration hadoopConfiguration) {
+    this.icebergCatalog = icebergCatalog;
+    this.deltaTableLocation = deltaTableLocation;
+    this.newTableIdentifier = newTableIdentifier;
+    this.hadoopConfiguration = hadoopConfiguration;
+    // TODO: check whether we can retrieve hadoopConfiguration directly
+    this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation);
+  }
+
+  @Override
+  public MigrateDeltaLakeTable tableProperties(Map<String, String> properties) {
+    additionalProperties.putAll(properties);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update();
+    Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema());
+    PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema);
+    // TODO: check whether we need more info when initializing the table
+    Table icebergTable =
+        this.icebergCatalog.createTable(
+            newTableIdentifier,
+            schema,
+            partitionSpec,
+            destTableProperties(
+                updatedSnapshot, this.deltaTableLocation, this.additionalProperties));
+
+    copyFromDeltaLakeToIceberg(icebergTable, partitionSpec);
+
+    Snapshot snapshot = icebergTable.currentSnapshot();
+    long totalDataFiles =
+        Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+    LOG.info(
+        "Successfully loaded Iceberg metadata for {} files to {}",
+        totalDataFiles,
+        deltaTableLocation);
+    return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles);
+  }
+
+  /** TODO: check the correctness for nested schema */
+  private Schema convertDeltaLakeSchemaToSchema(io.delta.standalone.types.StructType deltaSchema) {
+    Type converted =
+        DeltaLakeDataTypeVisitor.visit(deltaSchema, new DeltaLakeTypeToType(deltaSchema));
+    return new Schema(converted.asNestedType().asStructType().fields());
+  }
+
+  private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) {
+    List<String> partitionNames = deltaLog.snapshot().getMetadata().getPartitionColumns();
+    if (partitionNames.isEmpty()) {
+      return PartitionSpec.unpartitioned();
+    }
+
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+    for (String partitionName : partitionNames) {
+      builder.identity(partitionName);
+    }
+    return builder.build();
+  }
+
+  private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) {
+    // TODO: double check the arguments' meaning here
+    Iterator<VersionLog> it = deltaLog.getChanges(0, false);
+
+    while (it.hasNext()) {
+      VersionLog versionLog = it.next();
+      List<Action> actions = versionLog.getActions();
+
+      // We first need to iterate through to see what kind of transaction this was. There are 3
+      // cases:
+      // 1. AppendFile - when there are only AddFile instances (an INSERT on the table)
+      // 2. DeleteFiles - when there are only RemoveFile instances (a DELETE where all the records
+      // of file(s) were removed
+      // 3. OverwriteFiles - when there are a mix of AddFile and RemoveFile (a DELETE/UPDATE)
+
+      // Create a map of Delta Lake Action (AddFile, RemoveFile, etc.) --> List<Action>
+      Map<String, List<Action>> deltaLakeActionMap =
+          actions.stream()
+              .filter(action -> action instanceof AddFile || action instanceof RemoveFile)
+              .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName()));
+
+      // Scan the map so that we know what type of transaction this will be in Iceberg
+      IcebergTransactionType icebergTransactionType =
+          getIcebergTransactionTypeFromDeltaActions(deltaLakeActionMap);
+      if (icebergTransactionType == null) {
+        // TODO: my understanding here is that if the transaction type is undefined,
+        //  we can no longer continue even the next versionLog contains valid transaction type
+        //  may need further check
+        return;
+      }
+
+      List<DataFile> filesToAdd = Lists.newArrayList();
+      List<DataFile> filesToRemove = Lists.newArrayList();
+      for (Action action : Iterables.concat(deltaLakeActionMap.values())) {
+        DataFile dataFile = buildDataFileFromAction(action, table, spec);
+        if (action instanceof AddFile) {
+          filesToAdd.add(dataFile);
+        } else if (action instanceof RemoveFile) {
+          filesToRemove.add(dataFile);
+        } else {
+          // TODO: refactor this exception
+          throw new RuntimeException("Wrong action type");
+        }
+      }
+
+      switch (icebergTransactionType) {
+        case APPEND_FILES:
+          AppendFiles appendFiles = table.newAppend();
+          filesToAdd.forEach(appendFiles::appendFile);
+          appendFiles.commit();
+          break;
+        case DELETE_FILES:
+          DeleteFiles deleteFiles = table.newDelete();
+          filesToRemove.forEach(deleteFiles::deleteFile);
+          deleteFiles.commit();
+          break;
+        case OVERWRITE_FILES:
+          OverwriteFiles overwriteFiles = table.newOverwrite();
+          filesToAdd.forEach(overwriteFiles::addFile);
+          filesToRemove.forEach(overwriteFiles::deleteFile);
+          overwriteFiles.commit();
+          break;
+      }
+    }
+  }
+
+  @Nullable
+  private IcebergTransactionType getIcebergTransactionTypeFromDeltaActions(
+      Map<String, List<Action>> actionsMap) {
+    IcebergTransactionType icebergTransactionType;
+    if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && !actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.APPEND_FILES;
+    } else if (actionsMap.containsKey(RemoveFile.class.getSimpleName())
+        && !actionsMap.containsKey(AddFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.DELETE_FILES;
+    } else if (actionsMap.containsKey(AddFile.class.getSimpleName())
+        && actionsMap.containsKey(RemoveFile.class.getSimpleName())) {
+      icebergTransactionType = IcebergTransactionType.OVERWRITE_FILES;
+    } else {
+      // Some other type of transaction, we can ignore
+      return null;
+    }
+    return icebergTransactionType;
+  }
+
+  private DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) {
+    String path;
+    long size;
+    Map<String, String> partitionValues;
+
+    if (action instanceof AddFile) {
+      AddFile addFile = (AddFile) action;
+      path = addFile.getPath();
+      size = addFile.getSize();
+      partitionValues = addFile.getPartitionValues();
+    } else if (action instanceof RemoveFile) {
+      RemoveFile removeFile = (RemoveFile) action;
+      path = removeFile.getPath();
+      size =
+          removeFile
+              .getSize()
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format("File %s removed with specifying a size", path)));
+      partitionValues =
+          Optional.ofNullable(removeFile.getPartitionValues())
+              .orElseThrow(
+                  () ->
+                      new RuntimeException(
+                          String.format(
+                              "File %s removed without specifying partition values", path)));
+    } else {
+      throw new IllegalStateException(
+          String.format(
+              "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName()));
+    }
+
+    String fullFilePath = deltaLog.getPath().toString() + File.separator + path;
+    FileFormat format = determineFileFormatFromPath(fullFilePath);
+
+    Metrics metrics = getMetricsForFile(table, fullFilePath, format);
+    String partition =
+        spec.fields().stream()
+            .map(PartitionField::name)
+            .map(name -> String.format("%s=%s", name, partitionValues.get(name)))
+            .collect(Collectors.joining("/"));
+
+    return DataFiles.builder(spec)
+        .withPath(fullFilePath)
+        .withFormat(format)
+        .withFileSizeInBytes(size)
+        .withMetrics(metrics)
+        .withPartitionPath(partition)
+        .withRecordCount(metrics.recordCount())
+        .build();
+  }
+
+  protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format);
+
+  private FileFormat determineFileFormatFromPath(String path) {
+    if (path.endsWith(parquetPostfix)) {
+      return FileFormat.PARQUET;
+    } else if (path.endsWith(avroPostfix)) {
+      return FileFormat.AVRO;
+    } else if (path.endsWith(orcPostfix)) {
+      return FileFormat.ORC;
+    } else {
+      throw new RuntimeException("The format of the file is unsupported: " + path);
+    }
+  }
+
+  private static Map<String, String> destTableProperties(
+      io.delta.standalone.Snapshot deltaSnapshot,
+      String tableLocation,
+      Map<String, String> additionalProperties) {
+    Map<String, String> properties = Maps.newHashMap();
+
+    properties.putAll(deltaSnapshot.getMetadata().getConfiguration());
+    properties.putAll(
+        ImmutableMap.of(
+            "provider",

Review Comment:
   provider is a very Spark specific thing, not sure if it should be added in core. It should be a part of the additional properties passed in from Spark engine.



-- 
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