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

[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6449: WIP: Delta, 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_r1061725464


##########
delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.delta;
+
+import io.delta.standalone.DeltaLog;
+import io.delta.standalone.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 org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.DeleteFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionField;
+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.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.delta.actions.BaseMigrateDeltaLakeTableActionResult;
+import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable;
+import org.apache.iceberg.delta.utils.DeltaLakeDataTypeVisitor;
+import org.apache.iceberg.delta.utils.DeltaLakeTypeToType;
+import org.apache.iceberg.delta.utils.FileMetricsReader;
+import org.apache.iceberg.exceptions.ValidationException;
+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;
+
+/**
+ * Takes a Delta Lake table's location and attempts to transform it into an Iceberg table in the
+ * same location with a different identifier.
+ */
+public class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class);
+  private static final String parquetSuffix = ".parquet";
+  private static final String avroSuffix = ".avro";
+  private static final String orcSuffix = ".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;
+    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 = convertDeltaLakeSchema(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 convertDeltaLakeSchema(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) {
+    Iterator<VersionLog> versionLogIterator =
+        deltaLog.getChanges(
+            0, // retrieve actions starting from the initial version
+            false); // not throw exception when data loss detected
+
+    while (versionLogIterator.hasNext()) {
+      VersionLog versionLog = versionLogIterator.next();

Review Comment:
   I think that could be an add-on feature later if there is such a need from community, but we don't need that now.



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