From 73e38e5dd17c13c81a8e959219fdc2009c0c165e Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 15:55:49 -0500 Subject: [PATCH 01/54] Cannot pass tests for unknown reason --- .../iceberg/actions/ActionsProvider.java | 5 + .../actions/MigrateDeltaLakeTable.java | 34 ++ build.gradle | 1 + .../BaseMigrateDeltaLakeTableAction.java | 312 ++++++++++++++++++ ...BaseMigrateDeltaLakeTableActionResult.java | 33 ++ .../iceberg/data/TableMigrationUtil.java | 4 +- spark/v3.3/build.gradle | 5 + .../apache/iceberg/spark/SparkSchemaUtil.java | 2 +- .../MigrateDeltaLakeLakeTableSparkAction.java | 187 +++++++++++ .../iceberg/spark/actions/SparkActions.java | 12 + .../actions/TestMigrateDeltaLakeTable.java | 208 ++++++++++++ versions.props | 2 + 12 files changed, 802 insertions(+), 3 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java create mode 100644 core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java create mode 100644 core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java create mode 100644 spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java create mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index e5b5766f918d..0179c3eb525f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -35,6 +35,11 @@ default MigrateTable migrateTable(String tableIdent) { this.getClass().getName() + " does not implement migrateTable"); } + default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement migrateDeltaLakeTable"); + } + /** Instantiates an action to delete orphan files. */ default DeleteOrphanFiles deleteOrphanFiles(Table table) { throw new UnsupportedOperationException( diff --git a/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java b/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java new file mode 100644 index 000000000000..ade5c981aca4 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.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; + +import java.util.Map; + +/** Migrates a Delta Lake table to Iceberg in place. */ +public interface MigrateDeltaLakeTable extends Action{ + + MigrateDeltaLakeTable tableProperties(Map properties); + interface Result { + /** Returns the number of imported data files. */ + long importedDataFilesCount(); + } +} diff --git a/build.gradle b/build.gradle index b40dba9489d4..cd5d221af635 100644 --- a/build.gradle +++ b/build.gradle @@ -293,6 +293,7 @@ project(':iceberg-core') { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.slf4j', module: 'slf4j-log4j12' } + compileOnly "io.delta:delta-standalone_2.12" testImplementation "org.eclipse.jetty:jetty-servlet" testImplementation "org.eclipse.jetty:jetty-server" diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java new file mode 100644 index 000000000000..6c8335e3e8ce --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -0,0 +1,312 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { + + private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class); + private final Map 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 properties) { + additionalProperties.putAll(properties); + return this; + } + + @Override + public Result execute() { + io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); + Schema schema = getSchemaFromDeltaSnapshot(updatedSnapshot); + 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); + + // TODO: can we do things similar to stageTable.commitStagedChanges in spark to avoid redundant + // IO? + + 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 whether toJson and fromJson works */ + private Schema getSchemaFromDeltaSnapshot(io.delta.standalone.Snapshot deltaSnapshot) { + io.delta.standalone.types.StructType deltaSchema = deltaSnapshot.getMetadata().getSchema(); + if (deltaSchema == null) { + throw new IllegalStateException("Could not find schema in existing Delta Lake table."); + } + return SchemaParser.fromJson(deltaSchema.toJson()); + } + + private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { + List 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(); + } + + protected void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { + // TODO: double check the arguments' meaning here + Iterator it = deltaLog.getChanges(0, false); + + while (it.hasNext()) { + VersionLog versionLog = it.next(); + List 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 + Map> 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 filesToAdd = Lists.newArrayList(); + List 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> 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; + } + + protected DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { + String path; + long size; + Map 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; + String partition = + spec.fields().stream() + .map(PartitionField::name) + .map(name -> String.format("%s=%s", name, partitionValues.get(name))) + .collect(Collectors.joining("/")); + + DataFiles.Builder dataFileBuilder = + DataFiles.builder(spec) + .withPath(fullFilePath) + .withFileSizeInBytes(size) + .withPartitionPath(partition); + + // TODO: check the file format later, may not be parquet + return buildDataFile(dataFileBuilder, table, FileFormat.PARQUET, fullFilePath); + } + + protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format); + + private DataFile buildDataFile( + DataFiles.Builder dataFileBuilder, Table table, FileFormat format, String fullFilePath) { + Metrics metrics = getMetricsForFile(table, fullFilePath, format); + + return dataFileBuilder + .withFormat(format) + .withMetrics(metrics) + .withRecordCount(metrics.recordCount()) + .build(); + } + + protected static Map destTableProperties( + io.delta.standalone.Snapshot deltaSnapshot, + String tableLocation, + Map additionalProperties) { + Map properties = Maps.newHashMap(); + + properties.putAll(deltaSnapshot.getMetadata().getConfiguration()); + properties.putAll( + ImmutableMap.of( + "provider", + "iceberg", + "migrated", + "true", + "table_type", + "iceberg", + "location", + tableLocation)); + properties.putAll(additionalProperties); + + return properties; + } + + protected DeltaLog deltaLog() { + return this.deltaLog; + } + + protected String deltaTableLocation() { + return deltaTableLocation; + } + + protected Map additionalProperties() { + return additionalProperties; + } + + private enum IcebergTransactionType { + APPEND_FILES, + DELETE_FILES, + OVERWRITE_FILES + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java b/core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java new file mode 100644 index 000000000000..39dd1b94f3a4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java @@ -0,0 +1,33 @@ +/* + * 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 importedDataFilesCount; + + public BaseMigrateDeltaLakeTableActionResult(long importedDataFilesCount) { + this.importedDataFilesCount = importedDataFilesCount; + } + + @Override + public long importedDataFilesCount() { + return importedDataFilesCount; + } +} diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index aa1885a31e8c..44ca6a51d6aa 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -161,8 +161,8 @@ private static Metrics getAvroMetrics(Path path, Configuration conf) { } } - private static Metrics getParquetMetrics( - Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { + public static Metrics getParquetMetrics( + Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { InputFile file = HadoopInputFile.fromPath(path, conf); return ParquetUtil.fileMetrics(file, metricsSpec, mapping); diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 6e97ed147208..76c951061250 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -76,6 +76,8 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation("org.apache.parquet:parquet-column") implementation("org.apache.parquet:parquet-hadoop") + compileOnly ("io.delta:delta-standalone_${scalaVersion}") + implementation("org.apache.orc:orc-core::nohive") { exclude group: 'org.apache.hadoop' exclude group: 'commons-lang' @@ -90,6 +92,9 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'com.google.code.findbugs', module: 'jsr305' } + // Needed to write Delta Lake tables for testing + testImplementation "io.delta:delta-core_${scalaVersion}" + testImplementation("org.apache.hadoop:hadoop-minicluster") { exclude group: 'org.apache.avro', module: 'avro' // to make sure netty libs only come from project(':iceberg-arrow') diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 6075aba7ac5f..dab1f58976d3 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -338,7 +338,7 @@ private static PartitionSpec identitySpec(Schema schema, Collection colu return identitySpec(schema, names); } - private static PartitionSpec identitySpec(Schema schema, List partitionNames) { + public static PartitionSpec identitySpec(Schema schema, List partitionNames) { if (partitionNames == null || partitionNames.isEmpty()) { return null; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java new file mode 100644 index 000000000000..09b824f714d6 --- /dev/null +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java @@ -0,0 +1,187 @@ +/* + * 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 java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +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.TableProperties; +import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.TableMigrationUtil; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.StagedSparkTable; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +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.collection.JavaConverters; + +public class MigrateDeltaLakeLakeTableSparkAction extends BaseMigrateDeltaLakeTableAction { + + private static final Logger LOG = LoggerFactory.getLogger(MigrateDeltaLakeLakeTableSparkAction.class); + private final SparkSession spark; + private final StagingTableCatalog destCatalog; + private final Identifier newIdentifier; + + public MigrateDeltaLakeLakeTableSparkAction( + SparkSession spark, + CatalogPlugin destCatalog, + String deltaTableLocation, + Identifier newIdentifier) { + super( + null, + deltaTableLocation, + TableIdentifier.of(newIdentifier.name()), + spark.sessionState().newHadoopConf()); + this.spark = spark; + this.destCatalog = checkDestinationCatalog(destCatalog); + this.newIdentifier = newIdentifier; + } + + @Override + public Result execute() { + io.delta.standalone.Snapshot updatedSnapshot = deltaLog().update(); + StructType structType = getStructTypeFromDeltaSnapshot(); + StagedSparkTable stagedTable = + stageDestTable( + updatedSnapshot, + deltaTableLocation(), + destCatalog, + newIdentifier, + structType, + additionalProperties()); + PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(structType); + Table icebergTable = stagedTable.table(); + copyFromDeltaLakeToIceberg(icebergTable, partitionSpec); + + stagedTable.commitStagedChanges(); + 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); + } + + protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + return TableMigrationUtil.getParquetMetrics( + new Path(fullFilePath), spark.sessionState().newHadoopConf(), metricsConfig, nameMapping); + } + + private StagingTableCatalog checkDestinationCatalog(CatalogPlugin catalog) { + + return (StagingTableCatalog) catalog; + } + + private PartitionSpec getPartitionSpecFromDeltaSnapshot(StructType structType) { + Schema schema = SparkSchemaUtil.convert(structType); + PartitionSpec spec = + SparkSchemaUtil.identitySpec( + schema, deltaLog().snapshot().getMetadata().getPartitionColumns()); + return spec == null ? PartitionSpec.unpartitioned() : spec; + } + + private StructType getStructTypeFromDeltaSnapshot() { + io.delta.standalone.types.StructField[] fields = + Optional.ofNullable(deltaLog().snapshot().getMetadata().getSchema()) + .map(io.delta.standalone.types.StructType::getFields) + .orElseThrow(() -> new RuntimeException("Cannot determine table schema!")); + + // Convert from Delta StructFields to Spark StructFields + return new StructType( + Arrays.stream(fields) + .map( + s -> + new StructField( + s.getName(), + DataType.fromJson(s.getDataType().toJson()), + s.isNullable(), + Metadata.fromJson(s.getMetadata().toString()))) + .toArray(StructField[]::new)); + } + + private static StagedSparkTable stageDestTable( + io.delta.standalone.Snapshot deltaSnapshot, + String tableLocation, + StagingTableCatalog destinationCatalog, + Identifier destIdentifier, + StructType structType, + Map additionalProperties) { + try { + Map props = + destTableProperties(deltaSnapshot, tableLocation, additionalProperties); + io.delta.standalone.types.StructType schema = deltaSnapshot.getMetadata().getSchema(); + if (schema == null) { + throw new IllegalStateException("Could not find schema in existing Delta Lake table."); + } + + Transform[] partitioning = getPartitioning(deltaSnapshot); + + return (StagedSparkTable) + destinationCatalog.stageCreate(destIdentifier, structType, partitioning, props); + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException( + "Cannot create table %s as the namespace does not exist", destIdentifier); + } catch (TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot create table %s as it already exists", destIdentifier); + } + } + + private static Transform[] getPartitioning(io.delta.standalone.Snapshot deltaSnapshot) { + return deltaSnapshot.getMetadata().getPartitionColumns().stream() + .map( + name -> + LogicalExpressions.identity( + LogicalExpressions.reference( + JavaConverters.asScalaBuffer(Collections.singletonList(name))))) + .toArray(Transform[]::new); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 8c886adf510e..d3d285a530f1 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.MigrateDeltaLakeTable; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -67,6 +68,17 @@ public MigrateTableSparkAction migrateTable(String tableIdent) { spark, catalogAndIdent.catalog(), catalogAndIdent.identifier()); } + @Override + public MigrateDeltaLakeTable migrateDeltaLakeTable( + String newTableIdentifier, String deltaTableLocation) { + String ctx = "migrate delta target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeLakeTableSparkAction( + spark, catalogAndIdent.catalog(), deltaTableLocation, catalogAndIdent.identifier()); + } + @Override public RewriteDataFilesSparkAction rewriteDataFiles(Table table) { return new RewriteDataFilesSparkAction(spark, table); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java new file mode 100644 index 000000000000..a4b0e36028f8 --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java @@ -0,0 +1,208 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; +import org.apache.iceberg.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.source.SimpleRecord; +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.CatalogExtension; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.delta.catalog.DeltaCatalog; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.Parameterized; + +public class TestMigrateDeltaLakeTable extends SparkCatalogTestBase { + private static final String NAMESPACE = "default"; + + private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + private String partitionedIdentifier; + private String unpartitionedIdentifier; + + @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] { + "delta", + DeltaCatalog.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 temp = new TemporaryFolder(); + @Rule public TemporaryFolder other = new TemporaryFolder(); + + private final String partitionedTableName = "partitioned_table"; + private final String unpartitionedTableName = "unpartitioned_table"; + + private final String defaultSparkCatalog = "spark_catalog"; + private String partitionedLocation; + private String unpartitionedLocation; + private final String type; + private TableCatalog catalog; + + private String catalogName; + + public TestMigrateDeltaLakeTable( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + spark + .conf() + .set("spark.sql.catalog." + defaultSparkCatalog, SparkSessionCatalog.class.getName()); + this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); + this.type = config.get("type"); + this.catalogName = catalogName; + } + + @Before + public void before() { + try { + File partitionedFolder = temp.newFolder(); + File unpartitionedFolder = other.newFolder(); + partitionedLocation = partitionedFolder.toURI().toString(); + unpartitionedLocation = unpartitionedFolder.toURI().toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + partitionedIdentifier = destName(partitionedTableName); + unpartitionedIdentifier = destName(unpartitionedTableName); + + CatalogExtension delta = + (CatalogExtension) spark.sessionState().catalogManager().catalog("delta"); + // This needs to be set, otherwise Delta operations fail as the catalog is designed to override + // the default catalog (spark_catalog). + delta.setDelegateCatalog(spark.sessionState().catalogManager().currentCatalog()); + + spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); + + // Create a partitioned and unpartitioned table, doing a few inserts on each + IntStream.range(0, 3) + .forEach( + i -> { + List record = + Lists.newArrayList(new SimpleRecord(i, ALPHABET.substring(i, i + 1))); + + Dataset df = spark.createDataFrame(record, SimpleRecord.class); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .partitionBy("id") + .option("path", partitionedLocation) + .saveAsTable(partitionedIdentifier); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .option("path", unpartitionedLocation) + .saveAsTable(unpartitionedIdentifier); + }); + + // Delete a record from the table + spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); + spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); + + // 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() throws IOException { + // Drop the hive table. + spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(partitionedTableName))); + spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(unpartitionedTableName))); + } + + @Test + public void testMigratePartitioned() { + // This will test the scenario that the user switches the configuration and sets the default + // catalog to be Iceberg + // AFTER they had made it Delta and written a delta table there + spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); + + catalogName = defaultSparkCatalog; + String newTableIdentifier = destName("iceberg_table"); + MigrateDeltaLakeTable.Result result = + SparkActions.get().migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); + + // Compare the results + List oldResults = spark.sql("SELECT * FROM " + partitionedIdentifier).collectAsList(); + List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); + + Assert.assertEquals(oldResults.size(), newResults.size()); + Assert.assertTrue(newResults.containsAll(oldResults)); + Assert.assertTrue(oldResults.containsAll(newResults)); + } + + @Test + public void testMigrateUnpartitioned() { + // This will test the scenario that the user switches the configuration and sets the default + // catalog to be Iceberg + // AFTER they had made it Delta and written a delta table there + spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); + + catalogName = defaultSparkCatalog; + String newTableIdentifier = destName("iceberg_table_unpartitioned"); + MigrateDeltaLakeTable.Result result = + SparkActions.get() + .migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation) + .execute(); + + // Compare the results + List oldResults = spark.sql("SELECT * FROM " + unpartitionedIdentifier).collectAsList(); + List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); + + Assert.assertEquals(oldResults.size(), newResults.size()); + Assert.assertTrue(newResults.containsAll(oldResults)); + Assert.assertTrue(oldResults.containsAll(newResults)); + } + + private String destName(String dest) { + if (catalogName.equals("spark_catalog")) { + return NAMESPACE + "." + catalogName + "_" + type + "_" + dest; + } else { + return catalogName + "." + NAMESPACE + "." + catalogName + "_" + type + "_" + dest; + } + } +} diff --git a/versions.props b/versions.props index 60e4035b50fe..f3dd30cc2c64 100644 --- a/versions.props +++ b/versions.props @@ -28,6 +28,7 @@ org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0 com.emc.ecs:object-client-bundle = 3.3.2 org.immutables:value = 2.9.2 +io.delta:delta-standalone_2.12 = 0.5.0 # test deps org.junit.vintage:junit-vintage-engine = 5.8.2 @@ -44,3 +45,4 @@ org.mock-server:mockserver-netty = 5.13.2 org.mock-server:mockserver-client-java = 5.13.2 com.esotericsoftware:kryo = 4.0.2 org.eclipse.jetty:* = 9.4.43.v20210629 +io.delta:delta-core_2.12 = 2.1.0 From 5544f453f0f70f7e7ed13566065c77cb7941d764 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 16:27:31 -0500 Subject: [PATCH 02/54] fix test config issue and formatting --- .../iceberg/actions/ActionsProvider.java | 2 +- .../actions/MigrateDeltaLakeTable.java | 46 +-- build.gradle | 1 + .../BaseMigrateDeltaLakeTableAction.java | 8 +- .../iceberg/data/TableMigrationUtil.java | 2 +- .../apache/iceberg/spark/SparkTypeToType.java | 4 +- .../iceberg/spark/SparkTypeVisitor.java | 4 +- ... => MigrateDeltaLakeTableSparkAction.java} | 23 +- .../iceberg/spark/actions/SparkActions.java | 20 +- .../apache/iceberg/spark/SparkTestBase.java | 2 + .../actions/TestMigrateDeltaLakeTable.java | 316 +++++++++--------- 11 files changed, 214 insertions(+), 214 deletions(-) rename spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/{MigrateDeltaLakeLakeTableSparkAction.java => MigrateDeltaLakeTableSparkAction.java} (92%) diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 0179c3eb525f..1c962c03b629 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -37,7 +37,7 @@ default MigrateTable migrateTable(String tableIdent) { default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { throw new UnsupportedOperationException( - this.getClass().getName() + " does not implement migrateDeltaLakeTable"); + this.getClass().getName() + " does not implement migrateDeltaLakeTable"); } /** Instantiates an action to delete orphan files. */ diff --git a/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java b/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java index ade5c981aca4..b3bb3146ea1f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java +++ b/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java @@ -1,34 +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 * - * * 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. + * 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; import java.util.Map; /** Migrates a Delta Lake table to Iceberg in place. */ -public interface MigrateDeltaLakeTable extends Action{ +public interface MigrateDeltaLakeTable + extends Action { + + MigrateDeltaLakeTable tableProperties(Map properties); + + interface Result { - MigrateDeltaLakeTable tableProperties(Map properties); - interface Result { - /** Returns the number of imported data files. */ - long importedDataFilesCount(); - } + /** Returns the number of imported data files. */ + long importedDataFilesCount(); + } } diff --git a/build.gradle b/build.gradle index cd5d221af635..e385c7de15a4 100644 --- a/build.gradle +++ b/build.gradle @@ -293,6 +293,7 @@ project(':iceberg-core') { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.slf4j', module: 'slf4j-log4j12' } + compileOnly "io.delta:delta-standalone_2.12" testImplementation "org.eclipse.jetty:jetty-servlet" diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index 6c8335e3e8ce..59dd0031fb9a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -79,7 +79,7 @@ public Result execute() { // TODO: check whether we need more info when initializing the table Table icebergTable = this.icebergCatalog.createTable( - newTableIdentifier, + newTableIdentifier, schema, partitionSpec, destTableProperties( @@ -271,9 +271,9 @@ private DataFile buildDataFile( } protected static Map destTableProperties( - io.delta.standalone.Snapshot deltaSnapshot, - String tableLocation, - Map additionalProperties) { + io.delta.standalone.Snapshot deltaSnapshot, + String tableLocation, + Map additionalProperties) { Map properties = Maps.newHashMap(); properties.putAll(deltaSnapshot.getMetadata().getConfiguration()); diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index 44ca6a51d6aa..c598d0501939 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -162,7 +162,7 @@ private static Metrics getAvroMetrics(Path path, Configuration conf) { } public static Metrics getParquetMetrics( - Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { + Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { InputFile file = HadoopInputFile.fromPath(path, conf); return ParquetUtil.fileMetrics(file, metricsSpec, mapping); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 17499736fbeb..c865dceb616b 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -42,7 +42,7 @@ import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.types.VarcharType; -class SparkTypeToType extends SparkTypeVisitor { +public class SparkTypeToType extends SparkTypeVisitor { private final StructType root; private int nextId = 0; @@ -50,7 +50,7 @@ class SparkTypeToType extends SparkTypeVisitor { this.root = null; } - SparkTypeToType(StructType root) { + public SparkTypeToType(StructType root) { this.root = root; // the root struct's fields use the first ids this.nextId = root.fields().length; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java index 1ef694263fa4..d276816d1853 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java @@ -27,8 +27,8 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.UserDefinedType; -class SparkTypeVisitor { - static T visit(DataType type, SparkTypeVisitor visitor) { +public class SparkTypeVisitor { + public static T visit(DataType type, SparkTypeVisitor visitor) { if (type instanceof StructType) { StructField[] fields = ((StructType) type).fields(); List fieldResults = Lists.newArrayListWithExpectedSize(fields.length); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java similarity index 92% rename from spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java rename to spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 09b824f714d6..16139efd1f57 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -23,16 +23,7 @@ import java.util.Map; import java.util.Optional; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.MetricsConfig; -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.TableProperties; +import org.apache.iceberg.*; import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.TableMigrationUtil; @@ -57,14 +48,20 @@ import org.slf4j.LoggerFactory; import scala.collection.JavaConverters; -public class MigrateDeltaLakeLakeTableSparkAction extends BaseMigrateDeltaLakeTableAction { +/** + * 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 extends BaseMigrateDeltaLakeTableAction { + + private static final Logger LOG = LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class); - private static final Logger LOG = LoggerFactory.getLogger(MigrateDeltaLakeLakeTableSparkAction.class); private final SparkSession spark; private final StagingTableCatalog destCatalog; private final Identifier newIdentifier; - public MigrateDeltaLakeLakeTableSparkAction( + MigrateDeltaLakeTableSparkAction( SparkSession spark, CatalogPlugin destCatalog, String deltaTableLocation, diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index d3d285a530f1..619f8eacfcca 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -68,16 +68,16 @@ public MigrateTableSparkAction migrateTable(String tableIdent) { spark, catalogAndIdent.catalog(), catalogAndIdent.identifier()); } - @Override - public MigrateDeltaLakeTable migrateDeltaLakeTable( - String newTableIdentifier, String deltaTableLocation) { - String ctx = "migrate delta target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeLakeTableSparkAction( - spark, catalogAndIdent.catalog(), deltaTableLocation, catalogAndIdent.identifier()); - } + @Override + public MigrateDeltaLakeTable migrateDeltaLakeTable( + String newTableIdentifier, String deltaTableLocation) { + String ctx = "migrate delta target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeTableSparkAction( + spark, catalogAndIdent.catalog(), deltaTableLocation, catalogAndIdent.identifier()); + } @Override public RewriteDataFilesSparkAction rewriteDataFiles(Table table) { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 27fcb50817b1..9c3de50f67ae 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -71,6 +71,8 @@ public static void startMetastoreAndSpark() { .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + // Needed for Delta Lake tests + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .enableHiveSupport() .getOrCreate(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java index a4b0e36028f8..d6020bc0ac51 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java @@ -44,165 +44,165 @@ import org.junit.runners.Parameterized; public class TestMigrateDeltaLakeTable extends SparkCatalogTestBase { - private static final String NAMESPACE = "default"; - - private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; - private String partitionedIdentifier; - private String unpartitionedIdentifier; - - @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] { - "delta", - DeltaCatalog.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 - ) - } - }; + private static final String NAMESPACE = "default"; + + private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + private String partitionedIdentifier; + private String unpartitionedIdentifier; + + @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] { + "delta", + DeltaCatalog.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 temp = new TemporaryFolder(); + @Rule public TemporaryFolder other = new TemporaryFolder(); + + private final String partitionedTableName = "partitioned_table"; + private final String unpartitionedTableName = "unpartitioned_table"; + + private final String defaultSparkCatalog = "spark_catalog"; + private String partitionedLocation; + private String unpartitionedLocation; + private final String type; + private TableCatalog catalog; + + private String catalogName; + + public TestMigrateDeltaLakeTable( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + spark + .conf() + .set("spark.sql.catalog." + defaultSparkCatalog, SparkSessionCatalog.class.getName()); + this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); + this.type = config.get("type"); + this.catalogName = catalogName; + } + + @Before + public void before() { + try { + File partitionedFolder = temp.newFolder(); + File unpartitionedFolder = other.newFolder(); + partitionedLocation = partitionedFolder.toURI().toString(); + unpartitionedLocation = unpartitionedFolder.toURI().toString(); + } catch (IOException e) { + throw new RuntimeException(e); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - @Rule public TemporaryFolder other = new TemporaryFolder(); - - private final String partitionedTableName = "partitioned_table"; - private final String unpartitionedTableName = "unpartitioned_table"; - - private final String defaultSparkCatalog = "spark_catalog"; - private String partitionedLocation; - private String unpartitionedLocation; - private final String type; - private TableCatalog catalog; - - private String catalogName; - - public TestMigrateDeltaLakeTable( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - spark - .conf() - .set("spark.sql.catalog." + defaultSparkCatalog, SparkSessionCatalog.class.getName()); - this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); - this.type = config.get("type"); - this.catalogName = catalogName; - } - - @Before - public void before() { - try { - File partitionedFolder = temp.newFolder(); - File unpartitionedFolder = other.newFolder(); - partitionedLocation = partitionedFolder.toURI().toString(); - unpartitionedLocation = unpartitionedFolder.toURI().toString(); - } catch (IOException e) { - throw new RuntimeException(e); - } - - partitionedIdentifier = destName(partitionedTableName); - unpartitionedIdentifier = destName(unpartitionedTableName); - - CatalogExtension delta = - (CatalogExtension) spark.sessionState().catalogManager().catalog("delta"); - // This needs to be set, otherwise Delta operations fail as the catalog is designed to override - // the default catalog (spark_catalog). - delta.setDelegateCatalog(spark.sessionState().catalogManager().currentCatalog()); - - spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); - spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); - - // Create a partitioned and unpartitioned table, doing a few inserts on each - IntStream.range(0, 3) - .forEach( - i -> { - List record = - Lists.newArrayList(new SimpleRecord(i, ALPHABET.substring(i, i + 1))); - - Dataset df = spark.createDataFrame(record, SimpleRecord.class); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .partitionBy("id") - .option("path", partitionedLocation) - .saveAsTable(partitionedIdentifier); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .option("path", unpartitionedLocation) - .saveAsTable(unpartitionedIdentifier); - }); - - // Delete a record from the table - spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); - spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); - - // 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() throws IOException { - // Drop the hive table. - spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(partitionedTableName))); - spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(unpartitionedTableName))); - } - - @Test - public void testMigratePartitioned() { - // This will test the scenario that the user switches the configuration and sets the default - // catalog to be Iceberg - // AFTER they had made it Delta and written a delta table there - spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); - - catalogName = defaultSparkCatalog; - String newTableIdentifier = destName("iceberg_table"); - MigrateDeltaLakeTable.Result result = - SparkActions.get().migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); - - // Compare the results - List oldResults = spark.sql("SELECT * FROM " + partitionedIdentifier).collectAsList(); - List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); - - Assert.assertEquals(oldResults.size(), newResults.size()); - Assert.assertTrue(newResults.containsAll(oldResults)); - Assert.assertTrue(oldResults.containsAll(newResults)); - } - - @Test - public void testMigrateUnpartitioned() { - // This will test the scenario that the user switches the configuration and sets the default - // catalog to be Iceberg - // AFTER they had made it Delta and written a delta table there - spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); - - catalogName = defaultSparkCatalog; - String newTableIdentifier = destName("iceberg_table_unpartitioned"); - MigrateDeltaLakeTable.Result result = - SparkActions.get() - .migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation) - .execute(); - - // Compare the results - List oldResults = spark.sql("SELECT * FROM " + unpartitionedIdentifier).collectAsList(); - List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); - - Assert.assertEquals(oldResults.size(), newResults.size()); - Assert.assertTrue(newResults.containsAll(oldResults)); - Assert.assertTrue(oldResults.containsAll(newResults)); - } - - private String destName(String dest) { - if (catalogName.equals("spark_catalog")) { - return NAMESPACE + "." + catalogName + "_" + type + "_" + dest; - } else { - return catalogName + "." + NAMESPACE + "." + catalogName + "_" + type + "_" + dest; - } + partitionedIdentifier = destName(partitionedTableName); + unpartitionedIdentifier = destName(unpartitionedTableName); + + CatalogExtension delta = + (CatalogExtension) spark.sessionState().catalogManager().catalog("delta"); + // This needs to be set, otherwise Delta operations fail as the catalog is designed to override + // the default catalog (spark_catalog). + delta.setDelegateCatalog(spark.sessionState().catalogManager().currentCatalog()); + + spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); + + // Create a partitioned and unpartitioned table, doing a few inserts on each + IntStream.range(0, 3) + .forEach( + i -> { + List record = + Lists.newArrayList(new SimpleRecord(i, ALPHABET.substring(i, i + 1))); + + Dataset df = spark.createDataFrame(record, SimpleRecord.class); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .partitionBy("id") + .option("path", partitionedLocation) + .saveAsTable(partitionedIdentifier); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .option("path", unpartitionedLocation) + .saveAsTable(unpartitionedIdentifier); + }); + + // Delete a record from the table + spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); + spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); + + // 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() throws IOException { + // Drop the hive table. + spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(partitionedTableName))); + spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(unpartitionedTableName))); + } + + @Test + public void testMigratePartitioned() { + // This will test the scenario that the user switches the configuration and sets the default + // catalog to be Iceberg + // AFTER they had made it Delta and written a delta table there + spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); + + catalogName = defaultSparkCatalog; + String newTableIdentifier = destName("iceberg_table"); + MigrateDeltaLakeTable.Result result = + SparkActions.get().migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); + + // Compare the results + List oldResults = spark.sql("SELECT * FROM " + partitionedIdentifier).collectAsList(); + List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); + + Assert.assertEquals(oldResults.size(), newResults.size()); + Assert.assertTrue(newResults.containsAll(oldResults)); + Assert.assertTrue(oldResults.containsAll(newResults)); + } + + @Test + public void testMigrateUnpartitioned() { + // This will test the scenario that the user switches the configuration and sets the default + // catalog to be Iceberg + // AFTER they had made it Delta and written a delta table there + spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); + + catalogName = defaultSparkCatalog; + String newTableIdentifier = destName("iceberg_table_unpartitioned"); + MigrateDeltaLakeTable.Result result = + SparkActions.get() + .migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation) + .execute(); + + // Compare the results + List oldResults = spark.sql("SELECT * FROM " + unpartitionedIdentifier).collectAsList(); + List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); + + Assert.assertEquals(oldResults.size(), newResults.size()); + Assert.assertTrue(newResults.containsAll(oldResults)); + Assert.assertTrue(oldResults.containsAll(newResults)); + } + + private String destName(String dest) { + if (catalogName.equals("spark_catalog")) { + return NAMESPACE + "." + catalogName + "_" + type + "_" + dest; + } else { + return catalogName + "." + NAMESPACE + "." + catalogName + "_" + type + "_" + dest; } + } } From b8b61199751632703ec64868c228631e6285c987 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 18:50:39 -0500 Subject: [PATCH 03/54] transfer icebergCatalog to parent class --- .../spark/actions/MigrateDeltaLakeTableSparkAction.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 16139efd1f57..5892888625d5 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -31,6 +31,7 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.StagedSparkTable; import org.apache.spark.sql.SparkSession; @@ -66,10 +67,10 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA CatalogPlugin destCatalog, String deltaTableLocation, Identifier newIdentifier) { - super( - null, + // TODO: need further test the correctness of the name + super(Spark3Util.loadIcebergCatalog(spark, spark.sessionState().catalogManager().currentCatalog().name()), deltaTableLocation, - TableIdentifier.of(newIdentifier.name()), + TableIdentifier.parse(newIdentifier.toString()), spark.sessionState().newHadoopConf()); this.spark = spark; this.destCatalog = checkDestinationCatalog(destCatalog); From 274560cbbaec8532d6f0eaea787626f63fa64216 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 18:51:01 -0500 Subject: [PATCH 04/54] fix formatting --- .../spark/actions/MigrateDeltaLakeTableSparkAction.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 5892888625d5..0261b3d74c7a 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -68,7 +68,9 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA String deltaTableLocation, Identifier newIdentifier) { // TODO: need further test the correctness of the name - super(Spark3Util.loadIcebergCatalog(spark, spark.sessionState().catalogManager().currentCatalog().name()), + super( + Spark3Util.loadIcebergCatalog( + spark, spark.sessionState().catalogManager().currentCatalog().name()), deltaTableLocation, TableIdentifier.parse(newIdentifier.toString()), spark.sessionState().newHadoopConf()); From 39e35418cf6da2e172642acc5b8710e1acf2701d Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 22:27:48 -0500 Subject: [PATCH 05/54] implement direct schema transformation and stop using spark context to do the migration --- .../BaseMigrateDeltaLakeTableAction.java | 23 ++- .../util/DeltaLakeDataTypeVisitor.java | 74 ++++++++ .../iceberg/util/DeltaLakeTypeToType.java | 158 ++++++++++++++++++ .../MigrateDeltaLakeTableSparkAction.java | 109 ------------ 4 files changed, 250 insertions(+), 114 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java create mode 100644 core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index 59dd0031fb9a..eaeb013a1a17 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -39,6 +39,9 @@ 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.apache.iceberg.util.DeltaLakeDataTypeVisitor; +import org.apache.iceberg.util.DeltaLakeTypeToType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,13 +102,23 @@ public Result execute() { return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles); } - /** TODO: Check whether toJson and fromJson works */ + /** TODO: check the correctness for nested schema*/ private Schema getSchemaFromDeltaSnapshot(io.delta.standalone.Snapshot deltaSnapshot) { io.delta.standalone.types.StructType deltaSchema = deltaSnapshot.getMetadata().getSchema(); - if (deltaSchema == null) { - throw new IllegalStateException("Could not find schema in existing Delta Lake table."); - } - return SchemaParser.fromJson(deltaSchema.toJson()); + io.delta.standalone.types.StructField[] fields = + Optional.ofNullable(deltaSchema) + .map(io.delta.standalone.types.StructType::getFields) + .orElseThrow(() -> new RuntimeException("Cannot determine table schema!")); + Schema icebergSchema = convertDeltaLakeStructToSchema(deltaSchema); + LOG.info("Usual delta schema {}", deltaSchema.toJson()); + LOG.info("converted iceberg schema {}", SchemaParser.toJson(icebergSchema)); + return icebergSchema; + } + + private Schema convertDeltaLakeStructToSchema(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) { diff --git a/core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java b/core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java new file mode 100644 index 000000000000..ddd28c8dd4d2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/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.util; + +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 { + public static T visit(DataType type, DeltaLakeDataTypeVisitor visitor) { + if (type instanceof StructType) { + StructField[] fields = ((StructType) type).getFields(); + List 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 fieldResults) { + return null; + } + + public T field(StructField field, T typeResult) { + return null; + } + + public T array(ArrayType array, T elementResult) { + return null; + } + + public T map(MapType map, T keyResult, T valueResult) { + return null; + } + + public T atomic(DataType atomic) { + return null; + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java b/core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java new file mode 100644 index 000000000000..24eb442cb6d1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java @@ -0,0 +1,158 @@ +/* + * 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.util; + +import io.delta.standalone.types.ArrayType; +import io.delta.standalone.types.BinaryType; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.ByteType; +import io.delta.standalone.types.DataType; +import io.delta.standalone.types.DateType; +import io.delta.standalone.types.DecimalType; +import io.delta.standalone.types.DoubleType; +import io.delta.standalone.types.FloatType; +import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.LongType; +import io.delta.standalone.types.MapType; +import io.delta.standalone.types.ShortType; +import io.delta.standalone.types.StringType; +import io.delta.standalone.types.StructField; +import io.delta.standalone.types.StructType; +import io.delta.standalone.types.TimestampType; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class DeltaLakeTypeToType extends DeltaLakeDataTypeVisitor { + private final StructType root; + private int nextId = 0; + + DeltaLakeTypeToType() { + this.root = null; + } + + public DeltaLakeTypeToType(StructType root) { + this.root = root; + this.nextId = root.getFields().length; + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @Override + @SuppressWarnings("ReferenceEquality") + public Type struct(StructType struct, List types) { + StructField[] fields = struct.getFields(); + List newFields = Lists.newArrayListWithExpectedSize(fields.length); + boolean isRoot = root == struct; + for (int i = 0; i < fields.length; i += 1) { + StructField field = fields[i]; + Type type = types.get(i); + + int id; + if (isRoot) { + // for new conversions, use ordinals for ids in the root struct + id = i; + } else { + id = getNextId(); + } + + // TODO: refactor const string + String doc = + field.getMetadata().contains("comment") + ? field.getMetadata().get("comment").toString() + : null; + + if (field.isNullable()) { + newFields.add(Types.NestedField.optional(id, field.getName(), type, doc)); + } else { + newFields.add(Types.NestedField.required(id, field.getName(), type, doc)); + } + } + + return Types.StructType.of(newFields); + } + + @Override + public Type field(StructField field, Type typeResult) { + return typeResult; + } + + @Override + public Type array(ArrayType array, Type elementType) { + if (array.containsNull()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + @Override + public Type map(MapType map, Type keyType, Type valueType) { + if (map.valueContainsNull()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + @Override + public Type atomic(DataType atomic) { + // TODO: pay attention to Delta's NullType, currently unhandled + if (atomic instanceof BooleanType) { + return Types.BooleanType.get(); + + } else if (atomic instanceof IntegerType + || atomic instanceof ShortType + || atomic instanceof ByteType) { + return Types.IntegerType.get(); + + } else if (atomic instanceof LongType) { + return Types.LongType.get(); + + } else if (atomic instanceof FloatType) { + return Types.FloatType.get(); + + } else if (atomic instanceof DoubleType) { + return Types.DoubleType.get(); + + } else if (atomic instanceof StringType) { + return Types.StringType.get(); + + } else if (atomic instanceof DateType) { + return Types.DateType.get(); + + } else if (atomic instanceof TimestampType) { + return Types.TimestampType.withZone(); + + } else if (atomic instanceof DecimalType) { + return Types.DecimalType.of( + ((DecimalType) atomic).getPrecision(), ((DecimalType) atomic).getScale()); + } else if (atomic instanceof BinaryType) { + return Types.BinaryType.get(); + } + + throw new UnsupportedOperationException("Not a supported type: " + atomic.getCatalogString()); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 0261b3d74c7a..0fa8b42590a6 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -18,36 +18,19 @@ */ package org.apache.iceberg.spark.actions; -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; import org.apache.hadoop.fs.Path; import org.apache.iceberg.*; -import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.TableMigrationUtil; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.source.StagedSparkTable; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; 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.collection.JavaConverters; /** * Takes a Delta Lake table and attempts to transform it into an Iceberg table in the same location @@ -79,33 +62,6 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA this.newIdentifier = newIdentifier; } - @Override - public Result execute() { - io.delta.standalone.Snapshot updatedSnapshot = deltaLog().update(); - StructType structType = getStructTypeFromDeltaSnapshot(); - StagedSparkTable stagedTable = - stageDestTable( - updatedSnapshot, - deltaTableLocation(), - destCatalog, - newIdentifier, - structType, - additionalProperties()); - PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(structType); - Table icebergTable = stagedTable.table(); - copyFromDeltaLakeToIceberg(icebergTable, partitionSpec); - - stagedTable.commitStagedChanges(); - 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); - } - protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { MetricsConfig metricsConfig = MetricsConfig.forTable(table); String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); @@ -119,69 +75,4 @@ private StagingTableCatalog checkDestinationCatalog(CatalogPlugin catalog) { return (StagingTableCatalog) catalog; } - - private PartitionSpec getPartitionSpecFromDeltaSnapshot(StructType structType) { - Schema schema = SparkSchemaUtil.convert(structType); - PartitionSpec spec = - SparkSchemaUtil.identitySpec( - schema, deltaLog().snapshot().getMetadata().getPartitionColumns()); - return spec == null ? PartitionSpec.unpartitioned() : spec; - } - - private StructType getStructTypeFromDeltaSnapshot() { - io.delta.standalone.types.StructField[] fields = - Optional.ofNullable(deltaLog().snapshot().getMetadata().getSchema()) - .map(io.delta.standalone.types.StructType::getFields) - .orElseThrow(() -> new RuntimeException("Cannot determine table schema!")); - - // Convert from Delta StructFields to Spark StructFields - return new StructType( - Arrays.stream(fields) - .map( - s -> - new StructField( - s.getName(), - DataType.fromJson(s.getDataType().toJson()), - s.isNullable(), - Metadata.fromJson(s.getMetadata().toString()))) - .toArray(StructField[]::new)); - } - - private static StagedSparkTable stageDestTable( - io.delta.standalone.Snapshot deltaSnapshot, - String tableLocation, - StagingTableCatalog destinationCatalog, - Identifier destIdentifier, - StructType structType, - Map additionalProperties) { - try { - Map props = - destTableProperties(deltaSnapshot, tableLocation, additionalProperties); - io.delta.standalone.types.StructType schema = deltaSnapshot.getMetadata().getSchema(); - if (schema == null) { - throw new IllegalStateException("Could not find schema in existing Delta Lake table."); - } - - Transform[] partitioning = getPartitioning(deltaSnapshot); - - return (StagedSparkTable) - destinationCatalog.stageCreate(destIdentifier, structType, partitioning, props); - } catch (org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException e) { - throw new NoSuchNamespaceException( - "Cannot create table %s as the namespace does not exist", destIdentifier); - } catch (TableAlreadyExistsException e) { - throw new AlreadyExistsException( - "Cannot create table %s as it already exists", destIdentifier); - } - } - - private static Transform[] getPartitioning(io.delta.standalone.Snapshot deltaSnapshot) { - return deltaSnapshot.getMetadata().getPartitionColumns().stream() - .map( - name -> - LogicalExpressions.identity( - LogicalExpressions.reference( - JavaConverters.asScalaBuffer(Collections.singletonList(name))))) - .toArray(Transform[]::new); - } } From 92f962cde29c274a1fc19808ae795260fb207528 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 22:39:28 -0500 Subject: [PATCH 06/54] refactor and simplify the implementation --- .../BaseMigrateDeltaLakeTableAction.java | 30 ++++------------- .../{util => }/DeltaLakeDataTypeVisitor.java | 2 +- .../{util => }/DeltaLakeTypeToType.java | 32 ++++++++++--------- .../MigrateDeltaLakeTableSparkAction.java | 1 - 4 files changed, 25 insertions(+), 40 deletions(-) rename core/src/main/java/org/apache/iceberg/{util => }/DeltaLakeDataTypeVisitor.java (98%) rename core/src/main/java/org/apache/iceberg/{util => }/DeltaLakeTypeToType.java (83%) diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index eaeb013a1a17..9de20565495a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -40,8 +40,6 @@ 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.apache.iceberg.util.DeltaLakeDataTypeVisitor; -import org.apache.iceberg.util.DeltaLakeTypeToType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,7 +75,7 @@ public MigrateDeltaLakeTable tableProperties(Map properties) { @Override public Result execute() { io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); - Schema schema = getSchemaFromDeltaSnapshot(updatedSnapshot); + Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); // TODO: check whether we need more info when initializing the table Table icebergTable = @@ -87,10 +85,8 @@ public Result execute() { partitionSpec, destTableProperties( updatedSnapshot, this.deltaTableLocation, this.additionalProperties)); - copyFromDeltaLakeToIceberg(icebergTable, partitionSpec); - // TODO: can we do things similar to stageTable.commitStagedChanges in spark to avoid redundant - // IO? + copyFromDeltaLakeToIceberg(icebergTable, partitionSpec); Snapshot snapshot = icebergTable.currentSnapshot(); long totalDataFiles = @@ -102,20 +98,8 @@ public Result execute() { return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles); } - /** TODO: check the correctness for nested schema*/ - private Schema getSchemaFromDeltaSnapshot(io.delta.standalone.Snapshot deltaSnapshot) { - io.delta.standalone.types.StructType deltaSchema = deltaSnapshot.getMetadata().getSchema(); - io.delta.standalone.types.StructField[] fields = - Optional.ofNullable(deltaSchema) - .map(io.delta.standalone.types.StructType::getFields) - .orElseThrow(() -> new RuntimeException("Cannot determine table schema!")); - Schema icebergSchema = convertDeltaLakeStructToSchema(deltaSchema); - LOG.info("Usual delta schema {}", deltaSchema.toJson()); - LOG.info("converted iceberg schema {}", SchemaParser.toJson(icebergSchema)); - return icebergSchema; - } - - private Schema convertDeltaLakeStructToSchema(io.delta.standalone.types.StructType deltaSchema) { + /** 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()); @@ -134,7 +118,7 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { return builder.build(); } - protected void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { + private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { // TODO: double check the arguments' meaning here Iterator it = deltaLog.getChanges(0, false); @@ -220,7 +204,7 @@ private IcebergTransactionType getIcebergTransactionTypeFromDeltaActions( return icebergTransactionType; } - protected DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { + private DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { String path; long size; Map partitionValues; @@ -283,7 +267,7 @@ private DataFile buildDataFile( .build(); } - protected static Map destTableProperties( + private static Map destTableProperties( io.delta.standalone.Snapshot deltaSnapshot, String tableLocation, Map additionalProperties) { diff --git a/core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java b/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java similarity index 98% rename from core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java rename to core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java index ddd28c8dd4d2..587aa9f62e49 100644 --- a/core/src/main/java/org/apache/iceberg/util/DeltaLakeDataTypeVisitor.java +++ b/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.util; +package org.apache.iceberg; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.DataType; diff --git a/core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java b/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java similarity index 83% rename from core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java rename to core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java index 24eb442cb6d1..c042263d13c1 100644 --- a/core/src/main/java/org/apache/iceberg/util/DeltaLakeTypeToType.java +++ b/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java @@ -1,22 +1,24 @@ /* - * 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 + * * 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. * - * 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.util; +package org.apache.iceberg; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 0fa8b42590a6..831def127824 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -50,7 +50,6 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA CatalogPlugin destCatalog, String deltaTableLocation, Identifier newIdentifier) { - // TODO: need further test the correctness of the name super( Spark3Util.loadIcebergCatalog( spark, spark.sessionState().catalogManager().currentCatalog().name()), From 033c997aa7a3ec3d600916e16a360f28d05a207d Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 22:49:55 -0500 Subject: [PATCH 07/54] no need to make spark utils public --- .../apache/iceberg/DeltaLakeTypeToType.java | 30 +++++++++---------- .../apache/iceberg/spark/SparkSchemaUtil.java | 2 +- .../apache/iceberg/spark/SparkTypeToType.java | 4 +-- .../iceberg/spark/SparkTypeVisitor.java | 4 +-- 4 files changed, 19 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java b/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java index c042263d13c1..46a95aff14d2 100644 --- a/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java +++ b/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java @@ -1,22 +1,20 @@ /* + * 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 * - * * 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. + * 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; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index dab1f58976d3..6075aba7ac5f 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -338,7 +338,7 @@ private static PartitionSpec identitySpec(Schema schema, Collection colu return identitySpec(schema, names); } - public static PartitionSpec identitySpec(Schema schema, List partitionNames) { + private static PartitionSpec identitySpec(Schema schema, List partitionNames) { if (partitionNames == null || partitionNames.isEmpty()) { return null; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index c865dceb616b..17499736fbeb 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -42,7 +42,7 @@ import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.types.VarcharType; -public class SparkTypeToType extends SparkTypeVisitor { +class SparkTypeToType extends SparkTypeVisitor { private final StructType root; private int nextId = 0; @@ -50,7 +50,7 @@ public class SparkTypeToType extends SparkTypeVisitor { this.root = null; } - public SparkTypeToType(StructType root) { + SparkTypeToType(StructType root) { this.root = root; // the root struct's fields use the first ids this.nextId = root.fields().length; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java index d276816d1853..1ef694263fa4 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java @@ -27,8 +27,8 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.UserDefinedType; -public class SparkTypeVisitor { - public static T visit(DataType type, SparkTypeVisitor visitor) { +class SparkTypeVisitor { + static T visit(DataType type, SparkTypeVisitor visitor) { if (type instanceof StructType) { StructField[] fields = ((StructType) type).fields(); List fieldResults = Lists.newArrayListWithExpectedSize(fields.length); From 681a32fcb4bd4d3b68f9aa4031bcee2a5306b096 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 23:11:05 -0500 Subject: [PATCH 08/54] simplify the implementation in Spark Action --- .../MigrateDeltaLakeTableSparkAction.java | 26 ++++++------------- .../iceberg/spark/actions/SparkActions.java | 7 +---- 2 files changed, 9 insertions(+), 24 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 831def127824..98a1314ef806 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -19,16 +19,18 @@ package org.apache.iceberg.spark.actions; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.*; +import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.TableMigrationUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; -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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,23 +44,16 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA private static final Logger LOG = LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class); private final SparkSession spark; - private final StagingTableCatalog destCatalog; - private final Identifier newIdentifier; MigrateDeltaLakeTableSparkAction( - SparkSession spark, - CatalogPlugin destCatalog, - String deltaTableLocation, - Identifier newIdentifier) { + SparkSession spark, String deltaTableLocation, String newTableIdentifier) { super( Spark3Util.loadIcebergCatalog( spark, spark.sessionState().catalogManager().currentCatalog().name()), deltaTableLocation, - TableIdentifier.parse(newIdentifier.toString()), + TableIdentifier.parse(newTableIdentifier), spark.sessionState().newHadoopConf()); this.spark = spark; - this.destCatalog = checkDestinationCatalog(destCatalog); - this.newIdentifier = newIdentifier; } protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { @@ -69,9 +64,4 @@ protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat return TableMigrationUtil.getParquetMetrics( new Path(fullFilePath), spark.sessionState().newHadoopConf(), metricsConfig, nameMapping); } - - private StagingTableCatalog checkDestinationCatalog(CatalogPlugin catalog) { - - return (StagingTableCatalog) catalog; - } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 619f8eacfcca..3f3606bcbf93 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -71,12 +71,7 @@ public MigrateTableSparkAction migrateTable(String tableIdent) { @Override public MigrateDeltaLakeTable migrateDeltaLakeTable( String newTableIdentifier, String deltaTableLocation) { - String ctx = "migrate delta target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeTableSparkAction( - spark, catalogAndIdent.catalog(), deltaTableLocation, catalogAndIdent.identifier()); + return new MigrateDeltaLakeTableSparkAction(spark, deltaTableLocation, newTableIdentifier); } @Override From 77bdb27a684975116c661054dc4cb409057c648a Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Sun, 18 Dec 2022 23:29:29 -0500 Subject: [PATCH 09/54] add support for scala 2.13 --- build.gradle | 4 +++- versions.props | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index e385c7de15a4..63301873d24c 100644 --- a/build.gradle +++ b/build.gradle @@ -48,6 +48,8 @@ plugins { id 'nebula.dependency-recommender' version '11.0.0' } +String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") + try { // apply these plugins in a try-catch block so that we can handle cases without .git directory apply plugin: 'com.palantir.git-version' @@ -294,7 +296,7 @@ project(':iceberg-core') { exclude group: 'org.slf4j', module: 'slf4j-log4j12' } - compileOnly "io.delta:delta-standalone_2.12" + compileOnly "io.delta:delta-standalone_${scalaVersion}" testImplementation "org.eclipse.jetty:jetty-servlet" testImplementation "org.eclipse.jetty:jetty-server" diff --git a/versions.props b/versions.props index f3dd30cc2c64..d2bec3595a86 100644 --- a/versions.props +++ b/versions.props @@ -28,7 +28,7 @@ org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0 com.emc.ecs:object-client-bundle = 3.3.2 org.immutables:value = 2.9.2 -io.delta:delta-standalone_2.12 = 0.5.0 +io.delta:delta-standalone_* = 0.5.0 # test deps org.junit.vintage:junit-vintage-engine = 5.8.2 @@ -45,4 +45,4 @@ org.mock-server:mockserver-netty = 5.13.2 org.mock-server:mockserver-client-java = 5.13.2 com.esotericsoftware:kryo = 4.0.2 org.eclipse.jetty:* = 9.4.43.v20210629 -io.delta:delta-core_2.12 = 2.1.0 +io.delta:delta-core_* = 2.1.0 From 3dd540a94b368f51e9dfe62b157418f8e0ba6544 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Mon, 19 Dec 2022 00:15:52 -0500 Subject: [PATCH 10/54] add format check for data files --- .../BaseMigrateDeltaLakeTableAction.java | 53 +++++++++---------- .../iceberg/data/TableMigrationUtil.java | 4 +- .../MigrateDeltaLakeTableSparkAction.java | 22 +++++++- 3 files changed, 46 insertions(+), 33 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index 9de20565495a..a30c44209eed 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -46,6 +46,9 @@ public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class); + private final String PARQUET_POSTFIX = ".parquet"; + private final String AVRO_POSTFIX = ".avro"; + private final String ORC_POSTFIX = ".orc"; private final Map additionalProperties = Maps.newHashMap(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; @@ -238,35 +241,39 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp } 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("/")); - DataFiles.Builder dataFileBuilder = - DataFiles.builder(spec) - .withPath(fullFilePath) - .withFileSizeInBytes(size) - .withPartitionPath(partition); - - // TODO: check the file format later, may not be parquet - return buildDataFile(dataFileBuilder, table, FileFormat.PARQUET, fullFilePath); - } - - protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format); - - private DataFile buildDataFile( - DataFiles.Builder dataFileBuilder, Table table, FileFormat format, String fullFilePath) { - Metrics metrics = getMetricsForFile(table, fullFilePath, format); - - return dataFileBuilder + 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(PARQUET_POSTFIX)) { + return FileFormat.PARQUET; + } else if (path.endsWith(AVRO_POSTFIX)) { + return FileFormat.AVRO; + } else if (path.endsWith(ORC_POSTFIX)) { + return FileFormat.ORC; + } else { + throw new RuntimeException("The format of the file is unsupported: " + path); + } + } + private static Map destTableProperties( io.delta.standalone.Snapshot deltaSnapshot, String tableLocation, @@ -289,18 +296,6 @@ private static Map destTableProperties( return properties; } - protected DeltaLog deltaLog() { - return this.deltaLog; - } - - protected String deltaTableLocation() { - return deltaTableLocation; - } - - protected Map additionalProperties() { - return additionalProperties; - } - private enum IcebergTransactionType { APPEND_FILES, DELETE_FILES, diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index c598d0501939..cd9ba6a7637d 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -151,7 +151,7 @@ public static List listPartition( } } - private static Metrics getAvroMetrics(Path path, Configuration conf) { + public static Metrics getAvroMetrics(Path path, Configuration conf) { try { InputFile file = HadoopInputFile.fromPath(path, conf); long rowCount = Avro.rowCount(file); @@ -171,7 +171,7 @@ public static Metrics getParquetMetrics( } } - private static Metrics getOrcMetrics( + public static Metrics getOrcMetrics( Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { return OrcMetrics.fromInputFile(HadoopInputFile.fromPath(path, conf), metricsSpec, mapping); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 98a1314ef806..82fbe239bd82 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -61,7 +61,25 @@ protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; - return TableMigrationUtil.getParquetMetrics( - new Path(fullFilePath), spark.sessionState().newHadoopConf(), metricsConfig, nameMapping); + + switch (format) { + case PARQUET: + return TableMigrationUtil.getParquetMetrics( + new Path(fullFilePath), + spark.sessionState().newHadoopConf(), + metricsConfig, + nameMapping); + case AVRO: + return TableMigrationUtil.getAvroMetrics( + new Path(fullFilePath), spark.sessionState().newHadoopConf()); + case ORC: + return TableMigrationUtil.getOrcMetrics( + new Path(fullFilePath), + spark.sessionState().newHadoopConf(), + metricsConfig, + nameMapping); + default: + throw new RuntimeException("Unsupported file format: " + format); + } } } From 27ece9347abc412e5e34dd35c73a4c662408cfc2 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Mon, 19 Dec 2022 00:24:33 -0500 Subject: [PATCH 11/54] fix naming issue --- .../iceberg/BaseMigrateDeltaLakeTableAction.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index a30c44209eed..18b7dcd4e1de 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -46,9 +46,9 @@ public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class); - private final String PARQUET_POSTFIX = ".parquet"; - private final String AVRO_POSTFIX = ".avro"; - private final String ORC_POSTFIX = ".orc"; + private final String parquetPostfix = ".parquet"; + private final String avroPostfix = ".avro"; + private final String orcPostfix = ".orc"; private final Map additionalProperties = Maps.newHashMap(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; @@ -263,11 +263,11 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format); private FileFormat determineFileFormatFromPath(String path) { - if (path.endsWith(PARQUET_POSTFIX)) { + if (path.endsWith(parquetPostfix)) { return FileFormat.PARQUET; - } else if (path.endsWith(AVRO_POSTFIX)) { + } else if (path.endsWith(avroPostfix)) { return FileFormat.AVRO; - } else if (path.endsWith(ORC_POSTFIX)) { + } else if (path.endsWith(orcPostfix)) { return FileFormat.ORC; } else { throw new RuntimeException("The format of the file is unsupported: " + path); From a9faabf43b04dd1cb1e3ff54e09a29a7f82a16d8 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 16:21:30 -0500 Subject: [PATCH 12/54] make delta type visitor abstract --- .../iceberg/DeltaLakeDataTypeVisitor.java | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java b/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java index 587aa9f62e49..965b3ed2b218 100644 --- a/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java +++ b/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java @@ -26,7 +26,7 @@ import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public class DeltaLakeDataTypeVisitor { +public abstract class DeltaLakeDataTypeVisitor { public static T visit(DataType type, DeltaLakeDataTypeVisitor visitor) { if (type instanceof StructType) { StructField[] fields = ((StructType) type).getFields(); @@ -52,23 +52,13 @@ public static T visit(DataType type, DeltaLakeDataTypeVisitor visitor) { } } - public T struct(StructType struct, List fieldResults) { - return null; - } + public abstract T struct(StructType struct, List fieldResults); - public T field(StructField field, T typeResult) { - return null; - } + public abstract T field(StructField field, T typeResult); - public T array(ArrayType array, T elementResult) { - return null; - } + public abstract T array(ArrayType array, T elementResult); - public T map(MapType map, T keyResult, T valueResult) { - return null; - } + public abstract T map(MapType map, T keyResult, T valueResult); - public T atomic(DataType atomic) { - return null; - } + public abstract T atomic(DataType atomic); } From 3982711ad0f383459d0618828689fc9bcd523ab8 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 16:41:24 -0500 Subject: [PATCH 13/54] fix typo and nit problems --- .../BaseMigrateDeltaLakeTableAction.java | 30 ++++++++----------- .../MigrateDeltaLakeTableSparkAction.java | 6 ++++ 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index 18b7dcd4e1de..cb4ff161d844 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -46,9 +46,9 @@ 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 String parquetSuffix = ".parquet"; + private final String avroSuffix = ".avro"; + private final String orcSuffix = ".orc"; private final Map additionalProperties = Maps.newHashMap(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; @@ -78,7 +78,7 @@ public MigrateDeltaLakeTable tableProperties(Map properties) { @Override public Result execute() { io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); - Schema schema = convertDeltaLakeSchemaToSchema(updatedSnapshot.getMetadata().getSchema()); + Schema schema = convertDeltaLakeSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); // TODO: check whether we need more info when initializing the table Table icebergTable = @@ -102,7 +102,7 @@ public Result execute() { } /** TODO: check the correctness for nested schema */ - private Schema convertDeltaLakeSchemaToSchema(io.delta.standalone.types.StructType deltaSchema) { + private Schema convertDeltaLakeSchema(io.delta.standalone.types.StructType deltaSchema) { Type converted = DeltaLakeDataTypeVisitor.visit(deltaSchema, new DeltaLakeTypeToType(deltaSchema)); return new Schema(converted.asNestedType().asStructType().fields()); @@ -123,7 +123,10 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { // TODO: double check the arguments' meaning here - Iterator it = deltaLog.getChanges(0, false); + Iterator it = + deltaLog.getChanges( + 0, // retrieve actions from the initial version + false); // not throw exception when data loss detected while (it.hasNext()) { VersionLog versionLog = it.next(); @@ -263,11 +266,11 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format); private FileFormat determineFileFormatFromPath(String path) { - if (path.endsWith(parquetPostfix)) { + if (path.endsWith(parquetSuffix)) { return FileFormat.PARQUET; - } else if (path.endsWith(avroPostfix)) { + } else if (path.endsWith(avroSuffix)) { return FileFormat.AVRO; - } else if (path.endsWith(orcPostfix)) { + } else if (path.endsWith(orcSuffix)) { return FileFormat.ORC; } else { throw new RuntimeException("The format of the file is unsupported: " + path); @@ -283,14 +286,7 @@ private static Map destTableProperties( properties.putAll(deltaSnapshot.getMetadata().getConfiguration()); properties.putAll( ImmutableMap.of( - "provider", - "iceberg", - "migrated", - "true", - "table_type", - "iceberg", - "location", - tableLocation)); + "migration_source", "delta", "table_type", "iceberg", "location", tableLocation)); properties.putAll(additionalProperties); return properties; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 82fbe239bd82..968431374c4c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -18,7 +18,9 @@ */ package org.apache.iceberg.spark.actions; +import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.shaded.com.google.common.collect.ImmutableMap; import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Metrics; @@ -29,6 +31,7 @@ import org.apache.iceberg.data.TableMigrationUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; @@ -54,6 +57,9 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA TableIdentifier.parse(newTableIdentifier), spark.sessionState().newHadoopConf()); this.spark = spark; + Map properties = Maps.newHashMap(); + properties.putAll(ImmutableMap.of("provider", "iceberg")); + this.tableProperties(properties); } protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { From 9a7c4435bb2c7572ec5a00bf56c6d304c84989f9 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 17:34:33 -0500 Subject: [PATCH 14/54] migrate from iceberg-core to delta-lake --- build.gradle | 22 +++++++++++++++++-- .../BaseMigrateDeltaLakeTableAction.java | 0 .../iceberg/DeltaLakeDataTypeVisitor.java | 0 .../apache/iceberg/DeltaLakeTypeToType.java | 0 settings.gradle | 2 ++ spark/v3.3/build.gradle | 1 + versions.props | 2 +- 7 files changed, 24 insertions(+), 3 deletions(-) rename {core => delta-lake}/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java (100%) rename {core => delta-lake}/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java (100%) rename {core => delta-lake}/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java (100%) diff --git a/build.gradle b/build.gradle index 63301873d24c..1a0ff1f1dc7c 100644 --- a/build.gradle +++ b/build.gradle @@ -296,8 +296,6 @@ project(':iceberg-core') { exclude group: 'org.slf4j', module: 'slf4j-log4j12' } - compileOnly "io.delta:delta-standalone_${scalaVersion}" - testImplementation "org.eclipse.jetty:jetty-servlet" testImplementation "org.eclipse.jetty:jetty-server" testImplementation 'org.mock-server:mockserver-netty' @@ -442,6 +440,26 @@ project(':iceberg-aws') { } } +project(':iceberg-delta-lake') { + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + api project(':iceberg-api') + implementation project(':iceberg-common') + implementation project(':iceberg-core') + + compileOnly "io.delta:delta-standalone_${scalaVersion}" + + compileOnly("org.apache.hadoop:hadoop-common") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'javax.servlet', module: 'servlet-api' + exclude group: 'com.google.code.gson', module: 'gson' + } + + + } +} + project(':iceberg-gcp') { dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') diff --git a/core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java similarity index 100% rename from core/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java rename to delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java diff --git a/core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java b/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java similarity index 100% rename from core/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java rename to delta-lake/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java diff --git a/core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java similarity index 100% rename from core/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java rename to delta-lake/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java diff --git a/settings.gradle b/settings.gradle index d1a14abe5b0f..264c07864855 100644 --- a/settings.gradle +++ b/settings.gradle @@ -34,6 +34,7 @@ include 'hive-metastore' include 'nessie' include 'gcp' include 'dell' +include 'delta-lake' project(':api').name = 'iceberg-api' project(':common').name = 'iceberg-common' @@ -51,6 +52,7 @@ project(':hive-metastore').name = 'iceberg-hive-metastore' project(':nessie').name = 'iceberg-nessie' project(':gcp').name = 'iceberg-gcp' project(':dell').name = 'iceberg-dell' +project(':delta-lake').name = 'iceberg-delta-lake' if (null != System.getProperty("allVersions")) { System.setProperty("flinkVersions", System.getProperty("knownFlinkVersions")) diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 76c951061250..97ce5c54aab7 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -59,6 +59,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-orc') implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') + implementation project(':iceberg-delta-lake') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}") compileOnly "com.google.errorprone:error_prone_annotations" diff --git a/versions.props b/versions.props index d2bec3595a86..4864a4d38ff3 100644 --- a/versions.props +++ b/versions.props @@ -28,7 +28,7 @@ org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0 com.emc.ecs:object-client-bundle = 3.3.2 org.immutables:value = 2.9.2 -io.delta:delta-standalone_* = 0.5.0 +io.delta:delta-standalone_* = 0.6.0 # test deps org.junit.vintage:junit-vintage-engine = 5.8.2 From 173534e0ab98d205709eea807824aff281c8f1b3 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 18:16:16 -0500 Subject: [PATCH 15/54] move get Metrics for File to iceberg-delta-lake --- build.gradle | 2 ++ .../BaseMigrateDeltaLakeTableAction.java | 26 ++++++++++++-- .../MigrateDeltaLakeTableSparkAction.java | 36 ------------------- 3 files changed, 26 insertions(+), 38 deletions(-) diff --git a/build.gradle b/build.gradle index 1a0ff1f1dc7c..93dd7be5673e 100644 --- a/build.gradle +++ b/build.gradle @@ -446,6 +446,8 @@ project(':iceberg-delta-lake') { api project(':iceberg-api') implementation project(':iceberg-common') implementation project(':iceberg-core') + implementation project(':iceberg-data') + implementation "com.fasterxml.jackson.core:jackson-databind" compileOnly "io.delta:delta-standalone_${scalaVersion}" diff --git a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index cb4ff161d844..81c63a01bb6f 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -31,10 +31,14 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; 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.data.TableMigrationUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; 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; @@ -43,7 +47,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { +public class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class); private final String parquetSuffix = ".parquet"; @@ -263,7 +267,25 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp .build(); } - protected abstract Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format); + protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + + switch (format) { + case PARQUET: + return TableMigrationUtil.getParquetMetrics( + new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); + case AVRO: + return TableMigrationUtil.getAvroMetrics(new Path(fullFilePath), this.hadoopConfiguration); + case ORC: + return TableMigrationUtil.getOrcMetrics( + new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); + default: + throw new RuntimeException("Unsupported file format: " + format); + } + } private FileFormat determineFileFormatFromPath(String path) { if (path.endsWith(parquetSuffix)) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 968431374c4c..d764b6345e4f 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -19,18 +19,9 @@ package org.apache.iceberg.spark.actions; import java.util.Map; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.shaded.com.google.common.collect.ImmutableMap; import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.TableMigrationUtil; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; @@ -61,31 +52,4 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA properties.putAll(ImmutableMap.of("provider", "iceberg")); this.tableProperties(properties); } - - protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - NameMapping nameMapping = - nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; - - switch (format) { - case PARQUET: - return TableMigrationUtil.getParquetMetrics( - new Path(fullFilePath), - spark.sessionState().newHadoopConf(), - metricsConfig, - nameMapping); - case AVRO: - return TableMigrationUtil.getAvroMetrics( - new Path(fullFilePath), spark.sessionState().newHadoopConf()); - case ORC: - return TableMigrationUtil.getOrcMetrics( - new Path(fullFilePath), - spark.sessionState().newHadoopConf(), - metricsConfig, - nameMapping); - default: - throw new RuntimeException("Unsupported file format: " + format); - } - } } From bdd1ccf346b2b0b0c846c179c4b7aa0866626ca5 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 18:17:26 -0500 Subject: [PATCH 16/54] fix comment --- .../org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java index 81c63a01bb6f..7d4125a998e2 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java @@ -129,7 +129,7 @@ private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { // TODO: double check the arguments' meaning here Iterator it = deltaLog.getChanges( - 0, // retrieve actions from the initial version + 0, // retrieve actions starting from the initial version false); // not throw exception when data loss detected while (it.hasNext()) { From 85abac22ecd5e4c9698109deae17343dbea2f661 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Wed, 21 Dec 2022 18:38:07 -0500 Subject: [PATCH 17/54] fix wrong import --- .../iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index d764b6345e4f..aec1c6f4273e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.actions; import java.util.Map; -import org.apache.hadoop.shaded.com.google.common.collect.ImmutableMap; import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; 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.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; From 32e1af84d1d1bb046b11f34d896d189da5dfb93e Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Sat, 24 Dec 2022 01:37:02 -0500 Subject: [PATCH 18/54] Migrate delta to iceberg round 1 (#29) * remove redundant todo * move everything to iceberg-delta-lake and make the action a mixin called SupportMigrateDeltaLake * make constant string final and static * use filesToAdd/Remove to determine transaction directly * refactor and delete withRecordNumber from DataFiles.builder * refactor get partitionValues and use FileIO to get size when necessary * Add javadoc * refactor exceptions to be ValidationException * fix validationException format issuse * create new test base for spark delta test --- .../iceberg/actions/ActionsProvider.java | 5 - .../BaseMigrateDeltaLakeTableAction.java | 147 ++++++-------- .../{ => delta}/DeltaLakeDataTypeVisitor.java | 2 +- .../{ => delta}/DeltaLakeTypeToType.java | 2 +- ...BaseMigrateDeltaLakeTableActionResult.java | 12 +- .../delta}/actions/MigrateDeltaLakeTable.java | 17 +- .../actions/SupportMigrateDeltaLake.java | 32 +++ .../MigrateDeltaLakeTableSparkAction.java | 2 +- .../iceberg/spark/actions/SparkActions.java | 5 +- .../SparkDeltaLakeMigrationTestBase.java | 192 ++++++++++++++++++ .../apache/iceberg/spark/SparkTestBase.java | 2 - .../actions/TestMigrateDeltaLakeTable.java | 8 +- 12 files changed, 315 insertions(+), 111 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/{ => delta}/BaseMigrateDeltaLakeTableAction.java (71%) rename delta-lake/src/main/java/org/apache/iceberg/{ => delta}/DeltaLakeDataTypeVisitor.java (98%) rename delta-lake/src/main/java/org/apache/iceberg/{ => delta}/DeltaLakeTypeToType.java (99%) rename {core/src/main/java/org/apache/iceberg => delta-lake/src/main/java/org/apache/iceberg/delta}/actions/BaseMigrateDeltaLakeTableActionResult.java (76%) rename {api/src/main/java/org/apache/iceberg => delta-lake/src/main/java/org/apache/iceberg/delta}/actions/MigrateDeltaLakeTable.java (65%) create mode 100644 delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java create mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 1c962c03b629..e5b5766f918d 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -35,11 +35,6 @@ default MigrateTable migrateTable(String tableIdent) { this.getClass().getName() + " does not implement migrateTable"); } - default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { - throw new UnsupportedOperationException( - this.getClass().getName() + " does not implement migrateDeltaLakeTable"); - } - /** Instantiates an action to delete orphan files. */ default DeleteOrphanFiles deleteOrphanFiles(Table table) { throw new UnsupportedOperationException( diff --git a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java similarity index 71% rename from delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index 7d4125a998e2..e32fbfffa0fd 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg; +package org.apache.iceberg.delta; import io.delta.standalone.DeltaLog; import io.delta.standalone.VersionLog; @@ -29,14 +29,29 @@ 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.hadoop.fs.Path; -import org.apache.iceberg.actions.BaseMigrateDeltaLakeTableActionResult; -import org.apache.iceberg.actions.MigrateDeltaLakeTable; +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.MetricsConfig; +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.TableProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.TableMigrationUtil; +import org.apache.iceberg.delta.actions.BaseMigrateDeltaLakeTableActionResult; +import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -47,12 +62,16 @@ 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 final String parquetSuffix = ".parquet"; - private final String avroSuffix = ".avro"; - private final String orcSuffix = ".orc"; + private static final String parquetSuffix = ".parquet"; + private static final String avroSuffix = ".avro"; + private static final String orcSuffix = ".orc"; private final Map additionalProperties = Maps.newHashMap(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; @@ -69,7 +88,6 @@ public BaseMigrateDeltaLakeTableAction( 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); } @@ -126,14 +144,13 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { } private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { - // TODO: double check the arguments' meaning here - Iterator it = + Iterator versionLogIterator = deltaLog.getChanges( 0, // retrieve actions starting from the initial version false); // not throw exception when data loss detected - while (it.hasNext()) { - VersionLog versionLog = it.next(); + while (versionLogIterator.hasNext()) { + VersionLog versionLog = versionLogIterator.next(); List actions = versionLog.getActions(); // We first need to iterate through to see what kind of transaction this was. There are 3 @@ -149,16 +166,6 @@ private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { .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 filesToAdd = Lists.newArrayList(); List filesToRemove = Lists.newArrayList(); for (Action action : Iterables.concat(deltaLakeActionMap.values())) { @@ -168,85 +175,58 @@ private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { } else if (action instanceof RemoveFile) { filesToRemove.add(dataFile); } else { - // TODO: refactor this exception - throw new RuntimeException("Wrong action type"); + throw new ValidationException( + "The action %s's is unsupported", action.getClass().getSimpleName()); } } - 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; + if (filesToAdd.size() > 0 && filesToRemove.size() > 0) { + // Overwrite_Files case + OverwriteFiles overwriteFiles = table.newOverwrite(); + filesToAdd.forEach(overwriteFiles::addFile); + filesToRemove.forEach(overwriteFiles::deleteFile); + overwriteFiles.commit(); + } else if (filesToAdd.size() > 0) { + // Append_Files case + AppendFiles appendFiles = table.newAppend(); + filesToAdd.forEach(appendFiles::appendFile); + appendFiles.commit(); + } else if (filesToRemove.size() > 0) { + // Delete_Files case + DeleteFiles deleteFiles = table.newDelete(); + filesToRemove.forEach(deleteFiles::deleteFile); + deleteFiles.commit(); } } } - @Nullable - private IcebergTransactionType getIcebergTransactionTypeFromDeltaActions( - Map> 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; + Optional nullableSize; Map partitionValues; + long size; if (action instanceof AddFile) { AddFile addFile = (AddFile) action; path = addFile.getPath(); - size = addFile.getSize(); + nullableSize = Optional.of(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))); + nullableSize = removeFile.getSize(); + partitionValues = removeFile.getPartitionValues(); } else { throw new IllegalStateException( String.format( "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName())); } + if (partitionValues == null) { + // For unpartitioned table, the partitionValues should be an empty map rather than null + throw new ValidationException("File %s does not specify a partitionValues", path); + } + String fullFilePath = deltaLog.getPath().toString() + File.separator + path; FileFormat format = determineFileFormatFromPath(fullFilePath); @@ -257,13 +237,14 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp .map(name -> String.format("%s=%s", name, partitionValues.get(name))) .collect(Collectors.joining("/")); + size = nullableSize.orElseGet(() -> table.io().newInputFile(path).getLength()); + return DataFiles.builder(spec) .withPath(fullFilePath) .withFormat(format) .withFileSizeInBytes(size) .withMetrics(metrics) .withPartitionPath(partition) - .withRecordCount(metrics.recordCount()) .build(); } @@ -283,7 +264,7 @@ protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat return TableMigrationUtil.getOrcMetrics( new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); default: - throw new RuntimeException("Unsupported file format: " + format); + throw new ValidationException("Unsupported file format: %s", format); } } @@ -295,7 +276,7 @@ private FileFormat determineFileFormatFromPath(String path) { } else if (path.endsWith(orcSuffix)) { return FileFormat.ORC; } else { - throw new RuntimeException("The format of the file is unsupported: " + path); + throw new ValidationException("The format of the file %s is unsupported", path); } } @@ -313,10 +294,4 @@ private static Map destTableProperties( return properties; } - - private enum IcebergTransactionType { - APPEND_FILES, - DELETE_FILES, - OVERWRITE_FILES - } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java similarity index 98% rename from delta-lake/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java index 965b3ed2b218..ed7c3dd11e74 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeDataTypeVisitor.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg; +package org.apache.iceberg.delta; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.DataType; diff --git a/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java similarity index 99% rename from delta-lake/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java index 46a95aff14d2..df41820f7a24 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/DeltaLakeTypeToType.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg; +package org.apache.iceberg.delta; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java similarity index 76% rename from core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java index 39dd1b94f3a4..366ab867123e 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseMigrateDeltaLakeTableActionResult.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java @@ -16,18 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.actions; +package org.apache.iceberg.delta.actions; public class BaseMigrateDeltaLakeTableActionResult implements MigrateDeltaLakeTable.Result { - private final long importedDataFilesCount; + private final long migratedDataFilesCount; - public BaseMigrateDeltaLakeTableActionResult(long importedDataFilesCount) { - this.importedDataFilesCount = importedDataFilesCount; + public BaseMigrateDeltaLakeTableActionResult(long migratedDataFilesCount) { + this.migratedDataFilesCount = migratedDataFilesCount; } @Override - public long importedDataFilesCount() { - return importedDataFilesCount; + public long migratedDataFilesCount() { + return migratedDataFilesCount; } } diff --git a/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java similarity index 65% rename from api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java index b3bb3146ea1f..70380b7e675b 100644 --- a/api/src/main/java/org/apache/iceberg/actions/MigrateDeltaLakeTable.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java @@ -16,19 +16,28 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.actions; +package org.apache.iceberg.delta.actions; import java.util.Map; +import org.apache.iceberg.actions.Action; -/** Migrates a Delta Lake table to Iceberg in place. */ +/** Migrates an existing Delta Lake table to Iceberg in place. */ public interface MigrateDeltaLakeTable extends Action { + /** + * Sets table properties in the newly created Iceberg table. Any properties with the same key name + * will be overwritten. + * + * @param properties a map of properties to set + * @return this for method chaining + */ MigrateDeltaLakeTable tableProperties(Map properties); + /** The action result that contains a summary of the execution. */ interface Result { - /** Returns the number of imported data files. */ - long importedDataFilesCount(); + /** Returns the number of migrated data files. */ + long migratedDataFilesCount(); } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java new file mode 100644 index 000000000000..25b2046fdd7b --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java @@ -0,0 +1,32 @@ +/* + * 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.actions; + +/** + * An API that should be implemented by query engine integrations that want to support migration + * from Delta Lake table to Iceberg table. + */ +public interface SupportMigrateDeltaLake { + + /** Initiates an action to migrate an existing Delta Lake table to Iceberg. */ + default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement migrateDeltaLakeTable"); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index aec1c6f4273e..1917b22f5619 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -19,8 +19,8 @@ package org.apache.iceberg.spark.actions; import java.util.Map; -import org.apache.iceberg.BaseMigrateDeltaLakeTableAction; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.delta.BaseMigrateDeltaLakeTableAction; 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.Spark3Util; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 3f3606bcbf93..69c49baacb08 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,7 +20,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; -import org.apache.iceberg.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.delta.actions.SupportMigrateDeltaLake; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -32,7 +33,7 @@ *

This class is the primary API for interacting with actions in Spark that users should use to * instantiate particular actions. */ -public class SparkActions implements ActionsProvider { +public class SparkActions implements SupportMigrateDeltaLake, ActionsProvider { private final SparkSession spark; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java new file mode 100644 index 000000000000..8ea9c67f7851 --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java @@ -0,0 +1,192 @@ +/* + * 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; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +public abstract class SparkDeltaLakeMigrationTestBase { + protected static final Object ANY = new Object(); + + protected static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static SparkSession spark = null; + protected static HiveCatalog catalog = null; + + private static File warehouse = null; + + @BeforeClass + public static void startMetastoreAndSpark() { + SparkDeltaLakeMigrationTestBase.metastore = new TestHiveMetastore(); + metastore.start(); + SparkDeltaLakeMigrationTestBase.hiveConf = metastore.hiveConf(); + + SparkDeltaLakeMigrationTestBase.spark = + SparkSession.builder() + .master("local[2]") + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + // Needed for Delta Lake tests + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .enableHiveSupport() + .getOrCreate(); + + SparkDeltaLakeMigrationTestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterClass + public static void stopMetastoreAndSpark() throws Exception { + SparkDeltaLakeMigrationTestBase.catalog = null; + metastore.stop(); + SparkDeltaLakeMigrationTestBase.metastore = null; + spark.stop(); + SparkDeltaLakeMigrationTestBase.spark = null; + } + + @BeforeClass + public static void createWarehouse() throws IOException { + SparkDeltaLakeMigrationTestBase.warehouse = File.createTempFile("warehouse", null); + Assert.assertTrue(warehouse.delete()); + } + + @AfterClass + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + } + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + protected final String catalogName; + protected final Catalog validationCatalog; + protected final SupportsNamespaces validationNamespaceCatalog; + protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); + protected final String tableName; + + public SparkDeltaLakeMigrationTestBase() { + this(SparkCatalogConfig.HADOOP); + } + + public SparkDeltaLakeMigrationTestBase(SparkCatalogConfig config) { + this(config.catalogName(), config.implementation(), config.properties()); + } + + public SparkDeltaLakeMigrationTestBase( + String catalogName, String implementation, Map config) { + this.catalogName = catalogName; + this.validationCatalog = + catalogName.equals("testhadoop") + ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) + : catalog; + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + + spark.conf().set("spark.sql.catalog." + catalogName, implementation); + config.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); + + if (config.get("type").equalsIgnoreCase("hadoop")) { + spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); + } + + this.tableName = + (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table"; + + sql("CREATE NAMESPACE IF NOT EXISTS default"); + } + + protected String tableName(String name) { + return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default." + name; + } + + protected List sql(String query, Object... args) { + List rows = spark.sql(String.format(query, args)).collectAsList(); + if (rows.size() < 1) { + return ImmutableList.of(); + } + + return rowsToJava(rows); + } + + protected List rowsToJava(List rows) { + return rows.stream().map(this::toJava).collect(Collectors.toList()); + } + + private Object[] toJava(Row row) { + return IntStream.range(0, row.size()) + .mapToObj( + pos -> { + if (row.isNullAt(pos)) { + return null; + } + + Object value = row.get(pos); + if (value instanceof Row) { + return toJava((Row) value); + } else if (value instanceof scala.collection.Seq) { + return row.getList(pos); + } else if (value instanceof scala.collection.Map) { + return row.getJavaMap(pos); + } else { + return value; + } + }) + .toArray(Object[]::new); + } +} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 9c3de50f67ae..27fcb50817b1 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -71,8 +71,6 @@ public static void startMetastoreAndSpark() { .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") - // Needed for Delta Lake tests - .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .enableHiveSupport() .getOrCreate(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java index d6020bc0ac51..1b96b4f5cfc4 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java @@ -23,10 +23,10 @@ import java.util.List; import java.util.Map; import java.util.stream.IntStream; -import org.apache.iceberg.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkDeltaLakeMigrationTestBase; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Dataset; @@ -41,9 +41,11 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -public class TestMigrateDeltaLakeTable extends SparkCatalogTestBase { +@RunWith(Parameterized.class) +public class TestMigrateDeltaLakeTable extends SparkDeltaLakeMigrationTestBase { private static final String NAMESPACE = "default"; private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; From ac1141d405a54de95f411f9432a0026fcfc914d1 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Sun, 25 Dec 2022 13:10:12 -0500 Subject: [PATCH 19/54] Migrate delta to iceberg util refactor (#30) * refactor the structure of the package * copy-pase the util methods from TableMigrationUtil --- build.gradle | 3 +- .../iceberg/data/TableMigrationUtil.java | 6 +- .../BaseMigrateDeltaLakeTableAction.java | 10 +-- .../SupportMigrateDeltaLake.java | 4 +- .../{ => utils}/DeltaLakeDataTypeVisitor.java | 2 +- .../{ => utils}/DeltaLakeTypeToType.java | 2 +- .../delta/utils/FileMetricsReader.java | 65 +++++++++++++++++++ .../iceberg/spark/actions/SparkActions.java | 2 +- 8 files changed, 82 insertions(+), 12 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/delta/{actions => }/SupportMigrateDeltaLake.java (93%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{ => utils}/DeltaLakeDataTypeVisitor.java (98%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{ => utils}/DeltaLakeTypeToType.java (99%) create mode 100644 delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java diff --git a/build.gradle b/build.gradle index 93dd7be5673e..d1d28b8f2533 100644 --- a/build.gradle +++ b/build.gradle @@ -446,7 +446,8 @@ project(':iceberg-delta-lake') { api project(':iceberg-api') implementation project(':iceberg-common') implementation project(':iceberg-core') - implementation project(':iceberg-data') + implementation project(':iceberg-parquet') + implementation project(':iceberg-orc') implementation "com.fasterxml.jackson.core:jackson-databind" compileOnly "io.delta:delta-standalone_${scalaVersion}" diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index cd9ba6a7637d..aa1885a31e8c 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -151,7 +151,7 @@ public static List listPartition( } } - public static Metrics getAvroMetrics(Path path, Configuration conf) { + private static Metrics getAvroMetrics(Path path, Configuration conf) { try { InputFile file = HadoopInputFile.fromPath(path, conf); long rowCount = Avro.rowCount(file); @@ -161,7 +161,7 @@ public static Metrics getAvroMetrics(Path path, Configuration conf) { } } - public static Metrics getParquetMetrics( + private static Metrics getParquetMetrics( Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { InputFile file = HadoopInputFile.fromPath(path, conf); @@ -171,7 +171,7 @@ public static Metrics getParquetMetrics( } } - public static Metrics getOrcMetrics( + private static Metrics getOrcMetrics( Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { return OrcMetrics.fromInputFile(HadoopInputFile.fromPath(path, conf), metricsSpec, mapping); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index e32fbfffa0fd..cfd06ebbb87f 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -48,9 +48,11 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.TableMigrationUtil; 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.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; @@ -256,12 +258,12 @@ protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat switch (format) { case PARQUET: - return TableMigrationUtil.getParquetMetrics( + return FileMetricsReader.getParquetMetrics( new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); case AVRO: - return TableMigrationUtil.getAvroMetrics(new Path(fullFilePath), this.hadoopConfiguration); + return FileMetricsReader.getAvroMetrics(new Path(fullFilePath), this.hadoopConfiguration); case ORC: - return TableMigrationUtil.getOrcMetrics( + return FileMetricsReader.getOrcMetrics( new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); default: throw new ValidationException("Unsupported file format: %s", format); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java similarity index 93% rename from delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java index 25b2046fdd7b..906b3c8f8e81 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/SupportMigrateDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java @@ -16,7 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.actions; +package org.apache.iceberg.delta; + +import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; /** * An API that should be implemented by query engine integrations that want to support migration diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java similarity index 98% rename from delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java index ed7c3dd11e74..412c6483a330 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta; +package org.apache.iceberg.delta.utils; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.DataType; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java similarity index 99% rename from delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java index df41820f7a24..ef964070e252 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta; +package org.apache.iceberg.delta.utils; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java new file mode 100644 index 000000000000..c0236e39e94b --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java @@ -0,0 +1,65 @@ +/* + * 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.utils; + +import java.io.UncheckedIOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.orc.OrcMetrics; +import org.apache.iceberg.parquet.ParquetUtil; + +public class FileMetricsReader { + + private FileMetricsReader() {} + + public static Metrics getAvroMetrics(Path path, Configuration conf) { + try { + InputFile file = HadoopInputFile.fromPath(path, conf); + long rowCount = Avro.rowCount(file); + return new Metrics(rowCount, null, null, null, null); + } catch (UncheckedIOException e) { + throw new RuntimeException("Unable to read Avro file: " + path, e); + } + } + + public static Metrics getParquetMetrics( + Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { + try { + InputFile file = HadoopInputFile.fromPath(path, conf); + return ParquetUtil.fileMetrics(file, metricsSpec, mapping); + } catch (UncheckedIOException e) { + throw new RuntimeException("Unable to read the metrics of the Parquet file: " + path, e); + } + } + + public static Metrics getOrcMetrics( + Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { + try { + return OrcMetrics.fromInputFile(HadoopInputFile.fromPath(path, conf), metricsSpec, mapping); + } catch (UncheckedIOException e) { + throw new RuntimeException("Unable to read the metrics of the Orc file: " + path, e); + } + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 69c49baacb08..351efc756882 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,8 +20,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.delta.SupportMigrateDeltaLake; import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; -import org.apache.iceberg.delta.actions.SupportMigrateDeltaLake; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; From 8e9b3fc5c66b2caac7d785dd6408de2d334bb4fb Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 27 Dec 2022 23:13:02 -0500 Subject: [PATCH 20/54] Migrate delta to iceberg refactor 1.5 (#31) * move getFileMetrics to FileMetricsReader * add unit tests for schema conversion --- .../BaseMigrateDeltaLakeTableAction.java | 28 +--- .../delta/utils/FileMetricsReader.java | 32 ++++- .../delta/utils/TestDeltaLakeTypeToType.java | 124 ++++++++++++++++++ 3 files changed, 155 insertions(+), 29 deletions(-) create mode 100644 delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index cfd06ebbb87f..3c520af5ef1c 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -30,14 +30,12 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; 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.MetricsConfig; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; @@ -45,7 +43,6 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.delta.actions.BaseMigrateDeltaLakeTableActionResult; @@ -54,8 +51,6 @@ import org.apache.iceberg.delta.utils.DeltaLakeTypeToType; import org.apache.iceberg.delta.utils.FileMetricsReader; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; 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; @@ -232,7 +227,8 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp String fullFilePath = deltaLog.getPath().toString() + File.separator + path; FileFormat format = determineFileFormatFromPath(fullFilePath); - Metrics metrics = getMetricsForFile(table, fullFilePath, format); + Metrics metrics = + FileMetricsReader.getMetricsForFile(table, fullFilePath, format, this.hadoopConfiguration); String partition = spec.fields().stream() .map(PartitionField::name) @@ -250,26 +246,6 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp .build(); } - protected Metrics getMetricsForFile(Table table, String fullFilePath, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - NameMapping nameMapping = - nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; - - switch (format) { - case PARQUET: - return FileMetricsReader.getParquetMetrics( - new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); - case AVRO: - return FileMetricsReader.getAvroMetrics(new Path(fullFilePath), this.hadoopConfiguration); - case ORC: - return FileMetricsReader.getOrcMetrics( - new Path(fullFilePath), this.hadoopConfiguration, metricsConfig, nameMapping); - default: - throw new ValidationException("Unsupported file format: %s", format); - } - } - private FileFormat determineFileFormatFromPath(String path) { if (path.endsWith(parquetSuffix)) { return FileFormat.PARQUET; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java index c0236e39e94b..5dc7a9792187 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java @@ -21,12 +21,17 @@ import java.io.UncheckedIOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Metrics; import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.OrcMetrics; import org.apache.iceberg.parquet.ParquetUtil; @@ -34,7 +39,28 @@ public class FileMetricsReader { private FileMetricsReader() {} - public static Metrics getAvroMetrics(Path path, Configuration conf) { + public static Metrics getMetricsForFile( + Table table, String fullFilePath, FileFormat format, Configuration hadoopConfiguration) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + + switch (format) { + case PARQUET: + return getParquetMetrics( + new Path(fullFilePath), hadoopConfiguration, metricsConfig, nameMapping); + case AVRO: + return getAvroMetrics(new Path(fullFilePath), hadoopConfiguration); + case ORC: + return getOrcMetrics( + new Path(fullFilePath), hadoopConfiguration, metricsConfig, nameMapping); + default: + throw new ValidationException("Unsupported file format: %s", format); + } + } + + private static Metrics getAvroMetrics(Path path, Configuration conf) { try { InputFile file = HadoopInputFile.fromPath(path, conf); long rowCount = Avro.rowCount(file); @@ -44,7 +70,7 @@ public static Metrics getAvroMetrics(Path path, Configuration conf) { } } - public static Metrics getParquetMetrics( + private static Metrics getParquetMetrics( Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { InputFile file = HadoopInputFile.fromPath(path, conf); @@ -54,7 +80,7 @@ public static Metrics getParquetMetrics( } } - public static Metrics getOrcMetrics( + private static Metrics getOrcMetrics( Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { try { return OrcMetrics.fromInputFile(HadoopInputFile.fromPath(path, conf), metricsSpec, mapping); diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java new file mode 100644 index 000000000000..f215badb0a22 --- /dev/null +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta.utils; + +import io.delta.standalone.types.ArrayType; +import io.delta.standalone.types.BinaryType; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DoubleType; +import io.delta.standalone.types.LongType; +import io.delta.standalone.types.MapType; +import io.delta.standalone.types.StringType; +import io.delta.standalone.types.StructType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestDeltaLakeTypeToType { + private static final String optionalBooleanType = "testNullableBoolType"; + + private static final String requiredBinaryType = "testRequiredBinaryType"; + + private static final String doubleArrayType = "testNullableArrayType"; + + private static final String innerAtomicSchema = "testInnerAtomicSchema"; + + private static final String stringLongMapType = "testStringLongMap"; + + private StructType deltaAtomicSchema; + + private StructType deltaNestedSchema; + + @Before + public void constructDeltaLakeSchema() { + deltaAtomicSchema = + new StructType() + .add(optionalBooleanType, new BooleanType()) + .add(requiredBinaryType, new BinaryType(), false); + deltaNestedSchema = + new StructType() + .add(innerAtomicSchema, deltaAtomicSchema) + .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) + .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false); + } + + @Test + public void testAtomicTypeConversion() { + Type converted = + DeltaLakeDataTypeVisitor.visit( + deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema)); + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + Assert.assertTrue( + "The BooleanType should be converted to BooleanType", + convertedSchema.findType(optionalBooleanType) instanceof Types.BooleanType); + Assert.assertTrue( + "The converted BooleanType field is optional", + convertedSchema.findField(optionalBooleanType).isOptional()); + Assert.assertTrue( + "The BinaryType is converted to BinaryType", + convertedSchema.findType(requiredBinaryType) instanceof Types.BinaryType); + Assert.assertTrue( + "The converted BinaryType field is required", + convertedSchema.findField(requiredBinaryType).isRequired()); + } + + @Test + public void testNestedTypeConversion() { + Type converted = + DeltaLakeDataTypeVisitor.visit( + deltaNestedSchema, new DeltaLakeTypeToType(deltaNestedSchema)); + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + Assert.assertTrue( + "The StructType is converted to StructType", + convertedSchema.findType(innerAtomicSchema) instanceof Types.StructType); + Assert.assertTrue( + "The converted StructType contains subfield BooleanType", + convertedSchema.findType(innerAtomicSchema).asStructType().fieldType(optionalBooleanType) + instanceof Types.BooleanType); + Assert.assertTrue( + "The converted StructType contains subfield BinaryType", + convertedSchema.findType(innerAtomicSchema).asStructType().fieldType(requiredBinaryType) + instanceof Types.BinaryType); + + Assert.assertTrue( + "The MapType is converted to MapType", + convertedSchema.findType(stringLongMapType) instanceof Types.MapType); + Assert.assertTrue( + "The converted MapType has key as StringType", + convertedSchema.findType(stringLongMapType).asMapType().keyType() + instanceof Types.StringType); + Assert.assertTrue( + "The converted MapType has value as LongType", + convertedSchema.findType(stringLongMapType).asMapType().valueType() + instanceof Types.LongType); + + Assert.assertTrue( + "The ArrayType is converted to ListType", + convertedSchema.findType(doubleArrayType) instanceof Types.ListType); + Assert.assertTrue( + "The converted ListType field is required", + convertedSchema.findField(doubleArrayType).isRequired()); + Assert.assertTrue( + "The converted ListType field contains optional doubleType element", + convertedSchema.findType(doubleArrayType).asListType().isElementOptional()); + } +} From 8a8adefdb46daa1748d04a4b3f45507705b95120 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 29 Dec 2022 00:37:23 -0500 Subject: [PATCH 21/54] use transaction, refactor structure, add optional newTableLocation, add tableProperty (#32) * use transaction to commit all changes once * add optional newTableLocation * simplify the datafile build process, remove FileMetricsReader * refactor package structure * remove unnecessary types * fix format issue * add tableProperty method --- .../BaseMigrateDeltaLakeTableAction.java | 214 ++++++++++++------ ...BaseMigrateDeltaLakeTableActionResult.java | 2 +- .../{actions => }/MigrateDeltaLakeTable.java | 12 +- .../delta/SupportMigrateDeltaLake.java | 2 - .../delta/utils/FileMetricsReader.java | 91 -------- .../MigrateDeltaLakeTableSparkAction.java | 6 - .../iceberg/spark/actions/SparkActions.java | 2 +- .../actions/TestMigrateDeltaLakeTable.java | 2 +- 8 files changed, 157 insertions(+), 174 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/delta/{actions => }/BaseMigrateDeltaLakeTableActionResult.java (96%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{actions => }/MigrateDeltaLakeTable.java (81%) delete mode 100644 delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index 3c520af5ef1c..538fb1d230f6 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -24,18 +24,20 @@ import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.RemoveFile; import java.io.File; +import java.io.UncheckedIOException; 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.hadoop.fs.Path; 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.MetricsConfig; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; @@ -43,14 +45,21 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.avro.Avro; 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.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.orc.OrcMetrics; +import org.apache.iceberg.parquet.ParquetUtil; 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; @@ -60,31 +69,52 @@ 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. + * Takes a Delta Lake table's location and attempts to transform it into an Iceberg table in an + * optional user-specified location (default to the Delta Lake table's 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 static final String MIGRATION_SOURCE_PROP = "migration_source"; + private static final String DELTA_SOURCE_VALUE = "delta"; + private static final String ORIGINAL_LOCATION_PROP = "original_location"; + private static final String PARQUET_SUFFIX = ".parquet"; + private static final String AVRO_SUFFIX = ".avro"; + private static final String ORC_SUFFIX = ".orc"; private final Map additionalProperties = Maps.newHashMap(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; private final String deltaTableLocation; private final TableIdentifier newTableIdentifier; private final Configuration hadoopConfiguration; + private final String newTableLocation; + + public BaseMigrateDeltaLakeTableAction( + Catalog icebergCatalog, + String deltaTableLocation, + TableIdentifier newTableIdentifier, + Configuration hadoopConfiguration) { + this.icebergCatalog = icebergCatalog; + this.deltaTableLocation = deltaTableLocation; + this.newTableIdentifier = newTableIdentifier; + this.hadoopConfiguration = hadoopConfiguration; + this.newTableLocation = deltaTableLocation; + this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation); + } public BaseMigrateDeltaLakeTableAction( Catalog icebergCatalog, String deltaTableLocation, TableIdentifier newTableIdentifier, + String newTableLocation, Configuration hadoopConfiguration) { this.icebergCatalog = icebergCatalog; this.deltaTableLocation = deltaTableLocation; this.newTableIdentifier = newTableIdentifier; this.hadoopConfiguration = hadoopConfiguration; + this.newTableLocation = newTableLocation; this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation); } @@ -94,17 +124,23 @@ public MigrateDeltaLakeTable tableProperties(Map properties) { return this; } + @Override + public MigrateDeltaLakeTable tableProperty(String name, String value) { + additionalProperties.put(name, value); + 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, + this.newTableLocation, destTableProperties( updatedSnapshot, this.deltaTableLocation, this.additionalProperties)); @@ -114,13 +150,12 @@ public Result execute() { long totalDataFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); LOG.info( - "Successfully loaded Iceberg metadata for {} files to {}", + "Successfully loaded Iceberg metadata for {} files in {}", 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)); @@ -141,6 +176,8 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { } private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { + // Make the migration process into one transaction + Transaction transaction = table.newTransaction(); Iterator versionLogIterator = deltaLog.getChanges( 0, // retrieve actions starting from the initial version @@ -148,70 +185,74 @@ private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { while (versionLogIterator.hasNext()) { VersionLog versionLog = versionLogIterator.next(); - List 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 - Map> deltaLakeActionMap = - actions.stream() - .filter(action -> action instanceof AddFile || action instanceof RemoveFile) - .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName())); - - List filesToAdd = Lists.newArrayList(); - List 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 { - throw new ValidationException( - "The action %s's is unsupported", action.getClass().getSimpleName()); - } - } + commitDeltaVersionLogToIcebergTransaction(versionLog, transaction, table, spec); + } + + // commit transaction once all dataFiles are registered. + transaction.commitTransaction(); + } + + private void commitDeltaVersionLogToIcebergTransaction( + VersionLog versionLog, Transaction transaction, Table table, PartitionSpec spec) { + List 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) - if (filesToAdd.size() > 0 && filesToRemove.size() > 0) { - // Overwrite_Files case - OverwriteFiles overwriteFiles = table.newOverwrite(); - filesToAdd.forEach(overwriteFiles::addFile); - filesToRemove.forEach(overwriteFiles::deleteFile); - overwriteFiles.commit(); - } else if (filesToAdd.size() > 0) { - // Append_Files case - AppendFiles appendFiles = table.newAppend(); - filesToAdd.forEach(appendFiles::appendFile); - appendFiles.commit(); - } else if (filesToRemove.size() > 0) { - // Delete_Files case - DeleteFiles deleteFiles = table.newDelete(); - filesToRemove.forEach(deleteFiles::deleteFile); - deleteFiles.commit(); + // Create a map of Delta Lake Action (AddFile, RemoveFile, etc.) --> List + Map> deltaLakeActionMap = + actions.stream() + .filter(action -> action instanceof AddFile || action instanceof RemoveFile) + .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName())); + + List filesToAdd = Lists.newArrayList(); + List 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 { + throw new ValidationException( + "The action %s's is unsupported", action.getClass().getSimpleName()); } } + + if (filesToAdd.size() > 0 && filesToRemove.size() > 0) { + // Overwrite_Files case + OverwriteFiles overwriteFiles = transaction.newOverwrite(); + filesToAdd.forEach(overwriteFiles::addFile); + filesToRemove.forEach(overwriteFiles::deleteFile); + overwriteFiles.commit(); + } else if (filesToAdd.size() > 0) { + // Append_Files case + AppendFiles appendFiles = transaction.newAppend(); + filesToAdd.forEach(appendFiles::appendFile); + appendFiles.commit(); + } else if (filesToRemove.size() > 0) { + // Delete_Files case + DeleteFiles deleteFiles = transaction.newDelete(); + filesToRemove.forEach(deleteFiles::deleteFile); + deleteFiles.commit(); + } } - private DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { + public DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { String path; - Optional nullableSize; Map partitionValues; - long size; if (action instanceof AddFile) { AddFile addFile = (AddFile) action; path = addFile.getPath(); - nullableSize = Optional.of(addFile.getSize()); partitionValues = addFile.getPartitionValues(); } else if (action instanceof RemoveFile) { RemoveFile removeFile = (RemoveFile) action; path = removeFile.getPath(); - nullableSize = removeFile.getSize(); partitionValues = removeFile.getPartitionValues(); } else { throw new IllegalStateException( @@ -226,48 +267,79 @@ private DataFile buildDataFileFromAction(Action action, Table table, PartitionSp String fullFilePath = deltaLog.getPath().toString() + File.separator + path; FileFormat format = determineFileFormatFromPath(fullFilePath); + FileIO io = table.io(); + InputFile file; + if (io != null) { + file = io.newInputFile(fullFilePath); + } else { + file = HadoopInputFile.fromPath(new Path(fullFilePath), this.hadoopConfiguration); + } + + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + Metrics metrics = getMetricsForFile(file, format, metricsConfig, nameMapping); - Metrics metrics = - FileMetricsReader.getMetricsForFile(table, fullFilePath, format, this.hadoopConfiguration); String partition = spec.fields().stream() .map(PartitionField::name) .map(name -> String.format("%s=%s", name, partitionValues.get(name))) .collect(Collectors.joining("/")); - size = nullableSize.orElseGet(() -> table.io().newInputFile(path).getLength()); - return DataFiles.builder(spec) .withPath(fullFilePath) .withFormat(format) - .withFileSizeInBytes(size) + .withFileSizeInBytes(file.getLength()) .withMetrics(metrics) .withPartitionPath(partition) .build(); } private FileFormat determineFileFormatFromPath(String path) { - if (path.endsWith(parquetSuffix)) { + if (path.endsWith(PARQUET_SUFFIX)) { return FileFormat.PARQUET; - } else if (path.endsWith(avroSuffix)) { + } else if (path.endsWith(AVRO_SUFFIX)) { return FileFormat.AVRO; - } else if (path.endsWith(orcSuffix)) { + } else if (path.endsWith(ORC_SUFFIX)) { return FileFormat.ORC; } else { throw new ValidationException("The format of the file %s is unsupported", path); } } + private Metrics getMetricsForFile( + InputFile file, FileFormat format, MetricsConfig metricsSpec, NameMapping mapping) { + try { + switch (format) { + case AVRO: + long rowCount = Avro.rowCount(file); + return new Metrics(rowCount, null, null, null, null); + case PARQUET: + return ParquetUtil.fileMetrics(file, metricsSpec, mapping); + case ORC: + return OrcMetrics.fromInputFile(file, metricsSpec, mapping); + default: + throw new ValidationException("Unsupported file format: %s", format); + } + } catch (UncheckedIOException e) { + throw new RuntimeException( + String.format( + "Unable to read the metrics of the %s file: %s", format.name(), file.location()), + e); + } + } + private static Map destTableProperties( io.delta.standalone.Snapshot deltaSnapshot, - String tableLocation, + String originalLocation, Map additionalProperties) { Map properties = Maps.newHashMap(); properties.putAll(deltaSnapshot.getMetadata().getConfiguration()); properties.putAll( ImmutableMap.of( - "migration_source", "delta", "table_type", "iceberg", "location", tableLocation)); + MIGRATION_SOURCE_PROP, DELTA_SOURCE_VALUE, ORIGINAL_LOCATION_PROP, originalLocation)); properties.putAll(additionalProperties); return properties; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java similarity index 96% rename from delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java index 366ab867123e..5e094b913fd9 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/BaseMigrateDeltaLakeTableActionResult.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.actions; +package org.apache.iceberg.delta; public class BaseMigrateDeltaLakeTableActionResult implements MigrateDeltaLakeTable.Result { diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java b/delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java similarity index 81% rename from delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java index 70380b7e675b..709dd0e35060 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/actions/MigrateDeltaLakeTable.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.actions; +package org.apache.iceberg.delta; import java.util.Map; import org.apache.iceberg.actions.Action; @@ -34,6 +34,16 @@ public interface MigrateDeltaLakeTable */ MigrateDeltaLakeTable tableProperties(Map properties); + /** + * Sets a table property in the newly created Iceberg table. Any properties with the same key will + * be overwritten. + * + * @param name a table property name + * @param value a table property value + * @return this for method chaining + */ + MigrateDeltaLakeTable tableProperty(String name, String value); + /** The action result that contains a summary of the execution. */ interface Result { diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java index 906b3c8f8e81..fe1d76790a80 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.delta; -import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; - /** * An API that should be implemented by query engine integrations that want to support migration * from Delta Lake table to Iceberg table. diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java b/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java deleted file mode 100644 index 5dc7a9792187..000000000000 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/FileMetricsReader.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.utils; - -import java.io.UncheckedIOException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.OrcMetrics; -import org.apache.iceberg.parquet.ParquetUtil; - -public class FileMetricsReader { - - private FileMetricsReader() {} - - public static Metrics getMetricsForFile( - Table table, String fullFilePath, FileFormat format, Configuration hadoopConfiguration) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - NameMapping nameMapping = - nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; - - switch (format) { - case PARQUET: - return getParquetMetrics( - new Path(fullFilePath), hadoopConfiguration, metricsConfig, nameMapping); - case AVRO: - return getAvroMetrics(new Path(fullFilePath), hadoopConfiguration); - case ORC: - return getOrcMetrics( - new Path(fullFilePath), hadoopConfiguration, metricsConfig, nameMapping); - default: - throw new ValidationException("Unsupported file format: %s", format); - } - } - - private static Metrics getAvroMetrics(Path path, Configuration conf) { - try { - InputFile file = HadoopInputFile.fromPath(path, conf); - long rowCount = Avro.rowCount(file); - return new Metrics(rowCount, null, null, null, null); - } catch (UncheckedIOException e) { - throw new RuntimeException("Unable to read Avro file: " + path, e); - } - } - - private static Metrics getParquetMetrics( - Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { - try { - InputFile file = HadoopInputFile.fromPath(path, conf); - return ParquetUtil.fileMetrics(file, metricsSpec, mapping); - } catch (UncheckedIOException e) { - throw new RuntimeException("Unable to read the metrics of the Parquet file: " + path, e); - } - } - - private static Metrics getOrcMetrics( - Path path, Configuration conf, MetricsConfig metricsSpec, NameMapping mapping) { - try { - return OrcMetrics.fromInputFile(HadoopInputFile.fromPath(path, conf), metricsSpec, mapping); - } catch (UncheckedIOException e) { - throw new RuntimeException("Unable to read the metrics of the Orc file: " + path, e); - } - } -} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 1917b22f5619..5f5c6e3c856d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -18,11 +18,8 @@ */ package org.apache.iceberg.spark.actions; -import java.util.Map; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.delta.BaseMigrateDeltaLakeTableAction; -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.Spark3Util; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; @@ -48,8 +45,5 @@ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableA TableIdentifier.parse(newTableIdentifier), spark.sessionState().newHadoopConf()); this.spark = spark; - Map properties = Maps.newHashMap(); - properties.putAll(ImmutableMap.of("provider", "iceberg")); - this.tableProperties(properties); } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 351efc756882..5e7561626d8e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,8 +20,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.delta.MigrateDeltaLakeTable; import org.apache.iceberg.delta.SupportMigrateDeltaLake; -import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java index 1b96b4f5cfc4..71b11c9a37c7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.Map; import java.util.stream.IntStream; -import org.apache.iceberg.delta.actions.MigrateDeltaLakeTable; +import org.apache.iceberg.delta.MigrateDeltaLakeTable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkDeltaLakeMigrationTestBase; From 6fbf7402c7fe48151634d3e711f54e070395afb1 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Thu, 29 Dec 2022 17:39:53 -0500 Subject: [PATCH 22/54] fix the potential path error due to ambiguous return value of AddFile.getPath --- .../BaseMigrateDeltaLakeTableAction.java | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index 538fb1d230f6..5fa82273b09d 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -260,12 +260,13 @@ public DataFile buildDataFileFromAction(Action action, Table table, PartitionSpe "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName())); } + String fullFilePath = getFullFilePath(path); + if (partitionValues == null) { // For unpartitioned table, the partitionValues should be an empty map rather than null - throw new ValidationException("File %s does not specify a partitionValues", path); + throw new ValidationException("File %s does not specify a partitionValues", fullFilePath); } - String fullFilePath = deltaLog.getPath().toString() + File.separator + path; FileFormat format = determineFileFormatFromPath(fullFilePath); FileIO io = table.io(); InputFile file; @@ -344,4 +345,20 @@ private static Map destTableProperties( return properties; } + + /** + * Get the full file path, the input {@code String} can be either a relative path or an absolute + * path of a data file in delta table + * + * @param path the return value of {@link AddFile#getPath()} or {@link RemoveFile#getPath()} + * (either absolute or relative) + */ + private String getFullFilePath(String path) { + String tableRoot = deltaLog.getPath().toString(); + if (path.startsWith(tableRoot)) { + return path; + } else { + return tableRoot + File.separator + path; + } + } } From 69671b97418100654a6641b629351a4f902d337d Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 29 Dec 2022 21:04:03 -0500 Subject: [PATCH 23/54] refactor getFullPath with unit tests, use newCreateTableTransaction, remove unnecessary parameters and try-catch (#33) * refactor getFullFilePath to be static and add test * refactor the interface name * use newCreateTableTransaction, remove redundant parameters in helper methods * remove unnecessary try, catch --- .../BaseMigrateDeltaLakeTableAction.java | 64 ++++++++----------- ...ava => SupportMigrationFromDeltaLake.java} | 2 +- .../delta/TestBaseMigrateDeltaLake.java | 42 ++++++++++++ .../iceberg/spark/actions/SparkActions.java | 4 +- 4 files changed, 73 insertions(+), 39 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/delta/{SupportMigrateDeltaLake.java => SupportMigrationFromDeltaLake.java} (96%) create mode 100644 delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index 5fa82273b09d..dab8b70b3f79 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -24,7 +24,6 @@ import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.RemoveFile; import java.io.File; -import java.io.UncheckedIOException; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -60,6 +59,7 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.OrcMetrics; import org.apache.iceberg.parquet.ParquetUtil; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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; @@ -135,8 +135,8 @@ public Result execute() { io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); Schema schema = convertDeltaLakeSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); - Table icebergTable = - this.icebergCatalog.createTable( + Transaction icebergTransaction = + this.icebergCatalog.newCreateTableTransaction( newTableIdentifier, schema, partitionSpec, @@ -144,8 +144,10 @@ public Result execute() { destTableProperties( updatedSnapshot, this.deltaTableLocation, this.additionalProperties)); - copyFromDeltaLakeToIceberg(icebergTable, partitionSpec); + copyFromDeltaLakeToIceberg(icebergTransaction); + icebergTransaction.commitTransaction(); + Table icebergTable = icebergCatalog.loadTable(newTableIdentifier); Snapshot snapshot = icebergTable.currentSnapshot(); long totalDataFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); @@ -175,9 +177,7 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { return builder.build(); } - private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { - // Make the migration process into one transaction - Transaction transaction = table.newTransaction(); + private void copyFromDeltaLakeToIceberg(Transaction transaction) { Iterator versionLogIterator = deltaLog.getChanges( 0, // retrieve actions starting from the initial version @@ -185,15 +185,12 @@ private void copyFromDeltaLakeToIceberg(Table table, PartitionSpec spec) { while (versionLogIterator.hasNext()) { VersionLog versionLog = versionLogIterator.next(); - commitDeltaVersionLogToIcebergTransaction(versionLog, transaction, table, spec); + commitDeltaVersionLogToIcebergTransaction(versionLog, transaction); } - - // commit transaction once all dataFiles are registered. - transaction.commitTransaction(); } private void commitDeltaVersionLogToIcebergTransaction( - VersionLog versionLog, Transaction transaction, Table table, PartitionSpec spec) { + VersionLog versionLog, Transaction transaction) { List actions = versionLog.getActions(); // We first need to iterate through to see what kind of transaction this was. There are 3 @@ -212,7 +209,7 @@ private void commitDeltaVersionLogToIcebergTransaction( List filesToAdd = Lists.newArrayList(); List filesToRemove = Lists.newArrayList(); for (Action action : Iterables.concat(deltaLakeActionMap.values())) { - DataFile dataFile = buildDataFileFromAction(action, table, spec); + DataFile dataFile = buildDataFileFromAction(action, transaction.table()); if (action instanceof AddFile) { filesToAdd.add(dataFile); } else if (action instanceof RemoveFile) { @@ -242,7 +239,8 @@ private void commitDeltaVersionLogToIcebergTransaction( } } - public DataFile buildDataFileFromAction(Action action, Table table, PartitionSpec spec) { + public DataFile buildDataFileFromAction(Action action, Table table) { + PartitionSpec spec = table.spec(); String path; Map partitionValues; @@ -260,7 +258,7 @@ public DataFile buildDataFileFromAction(Action action, Table table, PartitionSpe "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName())); } - String fullFilePath = getFullFilePath(path); + String fullFilePath = getFullFilePath(path, deltaLog.getPath().toString()); if (partitionValues == null) { // For unpartitioned table, the partitionValues should be an empty map rather than null @@ -311,23 +309,16 @@ private FileFormat determineFileFormatFromPath(String path) { private Metrics getMetricsForFile( InputFile file, FileFormat format, MetricsConfig metricsSpec, NameMapping mapping) { - try { - switch (format) { - case AVRO: - long rowCount = Avro.rowCount(file); - return new Metrics(rowCount, null, null, null, null); - case PARQUET: - return ParquetUtil.fileMetrics(file, metricsSpec, mapping); - case ORC: - return OrcMetrics.fromInputFile(file, metricsSpec, mapping); - default: - throw new ValidationException("Unsupported file format: %s", format); - } - } catch (UncheckedIOException e) { - throw new RuntimeException( - String.format( - "Unable to read the metrics of the %s file: %s", format.name(), file.location()), - e); + switch (format) { + case AVRO: + long rowCount = Avro.rowCount(file); + return new Metrics(rowCount, null, null, null, null); + case PARQUET: + return ParquetUtil.fileMetrics(file, metricsSpec, mapping); + case ORC: + return OrcMetrics.fromInputFile(file, metricsSpec, mapping); + default: + throw new ValidationException("Unsupported file format: %s", format); } } @@ -347,14 +338,15 @@ private static Map destTableProperties( } /** - * Get the full file path, the input {@code String} can be either a relative path or an absolute - * path of a data file in delta table + * Get the full file path, the input {@code String} path can be either a relative path or an + * absolute path of a data file in delta table * * @param path the return value of {@link AddFile#getPath()} or {@link RemoveFile#getPath()} * (either absolute or relative) + * @param tableRoot the root path of the delta table */ - private String getFullFilePath(String path) { - String tableRoot = deltaLog.getPath().toString(); + @VisibleForTesting + static String getFullFilePath(String path, String tableRoot) { if (path.startsWith(tableRoot)) { return path; } else { diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java similarity index 96% rename from delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java index fe1d76790a80..bead6769fada 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrateDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java @@ -22,7 +22,7 @@ * An API that should be implemented by query engine integrations that want to support migration * from Delta Lake table to Iceberg table. */ -public interface SupportMigrateDeltaLake { +public interface SupportMigrationFromDeltaLake { /** Initiates an action to migrate an existing Delta Lake table to Iceberg. */ default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java new file mode 100644 index 000000000000..932b9a30c81f --- /dev/null +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import java.io.File; +import org.junit.Assert; +import org.junit.Test; + +public class TestBaseMigrateDeltaLake { + + @Test + public void testGetFullFilePath() { + String fileName = "part-00000-12345678-1234-1234-1234-123456789012.parquet"; + String tableRoot = "s3://bucket/table"; + String relativeFilePath = "id=0" + File.separator + fileName; + String absoluteFilePath = tableRoot + File.separator + relativeFilePath; + Assert.assertEquals( + "If the input is an absolute path, it should be returned as is", + absoluteFilePath, + BaseMigrateDeltaLakeTableAction.getFullFilePath(absoluteFilePath, tableRoot)); + Assert.assertEquals( + "If the input is a relative path, it should be returned as an absolute path", + absoluteFilePath, + BaseMigrateDeltaLakeTableAction.getFullFilePath(relativeFilePath, tableRoot)); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 5e7561626d8e..8da9d3543c21 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -21,7 +21,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; import org.apache.iceberg.delta.MigrateDeltaLakeTable; -import org.apache.iceberg.delta.SupportMigrateDeltaLake; +import org.apache.iceberg.delta.SupportMigrationFromDeltaLake; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -33,7 +33,7 @@ *

This class is the primary API for interacting with actions in Spark that users should use to * instantiate particular actions. */ -public class SparkActions implements SupportMigrateDeltaLake, ActionsProvider { +public class SparkActions implements SupportMigrationFromDeltaLake, ActionsProvider { private final SparkSession spark; From e3138a679dcecee3061bdcba2f3a48b72d949111 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 3 Jan 2023 00:23:29 -0500 Subject: [PATCH 24/54] allow user to specify a custom location for migrated table, fix load error of icebergCatalog (#34) * modify build.gradle to remove unnecessary dependency * fix nit problem * pass real env test in a questionable manner * allow user to specify a custom location for migrated table * remove unnecessary logger * restore build.gradle for spark --- build.gradle | 2 -- .../delta/SupportMigrationFromDeltaLake.java | 6 ++++ spark/v3.3/build.gradle | 4 +-- .../MigrateDeltaLakeTableSparkAction.java | 28 ++++++++++++------- .../iceberg/spark/actions/SparkActions.java | 25 ++++++++++++++++- 5 files changed, 50 insertions(+), 15 deletions(-) diff --git a/build.gradle b/build.gradle index d1d28b8f2533..841f562e7217 100644 --- a/build.gradle +++ b/build.gradle @@ -458,8 +458,6 @@ project(':iceberg-delta-lake') { exclude group: 'javax.servlet', module: 'servlet-api' exclude group: 'com.google.code.gson', module: 'gson' } - - } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java index bead6769fada..b838d327445f 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java @@ -29,4 +29,10 @@ default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String de throw new UnsupportedOperationException( this.getClass().getName() + " does not implement migrateDeltaLakeTable"); } + + default MigrateDeltaLakeTable migrateDeltaLakeTable( + String tableIdent, String deltaS3Location, String newTableLocation) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement migrateDeltaLakeTable"); + } } diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 97ce5c54aab7..13061b9015e7 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -77,8 +77,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation("org.apache.parquet:parquet-column") implementation("org.apache.parquet:parquet-hadoop") - compileOnly ("io.delta:delta-standalone_${scalaVersion}") - implementation("org.apache.orc:orc-core::nohive") { exclude group: 'org.apache.hadoop' exclude group: 'commons-lang' @@ -93,6 +91,8 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'com.google.code.findbugs', module: 'jsr305' } + compileOnly "io.delta:delta-standalone_${scalaVersion}" + // Needed to write Delta Lake tables for testing testImplementation "io.delta:delta-core_${scalaVersion}" diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java index 5f5c6e3c856d..e9c7fc3f3af7 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java @@ -22,8 +22,6 @@ import org.apache.iceberg.delta.BaseMigrateDeltaLakeTableAction; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Takes a Delta Lake table and attempts to transform it into an Iceberg table in the same location @@ -31,19 +29,29 @@ * table will refer to the newly migrated Iceberg table. */ public class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableAction { - - private static final Logger LOG = LoggerFactory.getLogger(MigrateDeltaLakeTableSparkAction.class); - - private final SparkSession spark; + MigrateDeltaLakeTableSparkAction( + SparkSession spark, + String deltaTableLocation, + String newTableIdentifier, + String catalogName) { + super( + Spark3Util.loadIcebergCatalog(spark, catalogName), + deltaTableLocation, + TableIdentifier.parse(newTableIdentifier), + spark.sessionState().newHadoopConf()); + } MigrateDeltaLakeTableSparkAction( - SparkSession spark, String deltaTableLocation, String newTableIdentifier) { + SparkSession spark, + String deltaTableLocation, + String newTableIdentifier, + String catalogName, + String newTableLocation) { super( - Spark3Util.loadIcebergCatalog( - spark, spark.sessionState().catalogManager().currentCatalog().name()), + Spark3Util.loadIcebergCatalog(spark, catalogName), deltaTableLocation, TableIdentifier.parse(newTableIdentifier), + newTableLocation, spark.sessionState().newHadoopConf()); - this.spark = spark; } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 8da9d3543c21..fe67da5b2966 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -72,7 +72,30 @@ public MigrateTableSparkAction migrateTable(String tableIdent) { @Override public MigrateDeltaLakeTable migrateDeltaLakeTable( String newTableIdentifier, String deltaTableLocation) { - return new MigrateDeltaLakeTableSparkAction(spark, deltaTableLocation, newTableIdentifier); + String ctx = "delta lake migrate target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeTableSparkAction( + spark, + deltaTableLocation, + catalogAndIdent.identifier().toString(), + catalogAndIdent.catalog().name()); + } + + @Override + public MigrateDeltaLakeTable migrateDeltaLakeTable( + String newTableIdentifier, String deltaTableLocation, String newTableLocation) { + String ctx = "delta lake migrate target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeTableSparkAction( + spark, + deltaTableLocation, + catalogAndIdent.identifier().toString(), + catalogAndIdent.catalog().name(), + newTableLocation); } @Override From 2e8dfd07cd4ec83adeae3a5665aca9c6723d48cb Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Fri, 6 Jan 2023 13:19:53 -0500 Subject: [PATCH 25/54] Fix nit problems and optimize some implementation (#38) * refactor mixin order * fix nit problems * add null check to the constructor * let copyFromDeltaLakeToIceberg return the number of totalDataFiles directly * use ImmutableMap.Builder * fix the problem in getFullFilePath * use hadoopFileIO to read dataFile * make type conversion util package-private * fix format --- .../BaseMigrateDeltaLakeTableAction.java | 90 ++++++++----------- .../{utils => }/DeltaLakeDataTypeVisitor.java | 4 +- .../{utils => }/DeltaLakeTypeToType.java | 6 +- .../{utils => }/TestDeltaLakeTypeToType.java | 2 +- .../iceberg/spark/actions/SparkActions.java | 2 +- 5 files changed, 43 insertions(+), 61 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/delta/{utils => }/DeltaLakeDataTypeVisitor.java (96%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{utils => }/DeltaLakeTypeToType.java (96%) rename delta-lake/src/test/java/org/apache/iceberg/delta/{utils => }/TestDeltaLakeTypeToType.java (99%) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index dab8b70b3f79..b0ba78c9f305 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -24,12 +24,12 @@ import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.RemoveFile; import java.io.File; +import java.net.URI; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -41,7 +41,6 @@ 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.TableProperties; @@ -49,11 +48,8 @@ import org.apache.iceberg.avro.Avro; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.delta.utils.DeltaLakeDataTypeVisitor; -import org.apache.iceberg.delta.utils.DeltaLakeTypeToType; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; @@ -63,7 +59,6 @@ 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; @@ -83,13 +78,14 @@ public class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { private static final String PARQUET_SUFFIX = ".parquet"; private static final String AVRO_SUFFIX = ".avro"; private static final String ORC_SUFFIX = ".orc"; - private final Map additionalProperties = Maps.newHashMap(); + private final ImmutableMap.Builder additionalPropertiesBuilder = + ImmutableMap.builder(); private final DeltaLog deltaLog; private final Catalog icebergCatalog; private final String deltaTableLocation; private final TableIdentifier newTableIdentifier; - private final Configuration hadoopConfiguration; private final String newTableLocation; + private final HadoopFileIO hadoopFileIO; public BaseMigrateDeltaLakeTableAction( Catalog icebergCatalog, @@ -99,9 +95,9 @@ public BaseMigrateDeltaLakeTableAction( this.icebergCatalog = icebergCatalog; this.deltaTableLocation = deltaTableLocation; this.newTableIdentifier = newTableIdentifier; - this.hadoopConfiguration = hadoopConfiguration; this.newTableLocation = deltaTableLocation; - this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation); + this.deltaLog = DeltaLog.forTable(hadoopConfiguration, deltaTableLocation); + this.hadoopFileIO = new HadoopFileIO(hadoopConfiguration); } public BaseMigrateDeltaLakeTableAction( @@ -113,20 +109,20 @@ public BaseMigrateDeltaLakeTableAction( this.icebergCatalog = icebergCatalog; this.deltaTableLocation = deltaTableLocation; this.newTableIdentifier = newTableIdentifier; - this.hadoopConfiguration = hadoopConfiguration; - this.newTableLocation = newTableLocation; - this.deltaLog = DeltaLog.forTable(this.hadoopConfiguration, this.deltaTableLocation); + this.newTableLocation = newTableLocation == null ? deltaTableLocation : newTableLocation; + this.deltaLog = DeltaLog.forTable(hadoopConfiguration, deltaTableLocation); + this.hadoopFileIO = new HadoopFileIO(hadoopConfiguration); } @Override public MigrateDeltaLakeTable tableProperties(Map properties) { - additionalProperties.putAll(properties); + additionalPropertiesBuilder.putAll(properties); return this; } @Override public MigrateDeltaLakeTable tableProperty(String name, String value) { - additionalProperties.put(name, value); + additionalPropertiesBuilder.put(name, value); return this; } @@ -136,21 +132,15 @@ public Result execute() { Schema schema = convertDeltaLakeSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); Transaction icebergTransaction = - this.icebergCatalog.newCreateTableTransaction( + icebergCatalog.newCreateTableTransaction( newTableIdentifier, schema, partitionSpec, - this.newTableLocation, - destTableProperties( - updatedSnapshot, this.deltaTableLocation, this.additionalProperties)); + newTableLocation, + destTableProperties(updatedSnapshot, deltaTableLocation)); - copyFromDeltaLakeToIceberg(icebergTransaction); + long totalDataFiles = copyFromDeltaLakeToIceberg(icebergTransaction); icebergTransaction.commitTransaction(); - - Table icebergTable = icebergCatalog.loadTable(newTableIdentifier); - Snapshot snapshot = icebergTable.currentSnapshot(); - long totalDataFiles = - Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); LOG.info( "Successfully loaded Iceberg metadata for {} files in {}", totalDataFiles, @@ -177,7 +167,7 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { return builder.build(); } - private void copyFromDeltaLakeToIceberg(Transaction transaction) { + private long copyFromDeltaLakeToIceberg(Transaction transaction) { Iterator versionLogIterator = deltaLog.getChanges( 0, // retrieve actions starting from the initial version @@ -187,6 +177,9 @@ private void copyFromDeltaLakeToIceberg(Transaction transaction) { VersionLog versionLog = versionLogIterator.next(); commitDeltaVersionLogToIcebergTransaction(versionLog, transaction); } + + return Long.parseLong( + transaction.table().currentSnapshot().summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); } private void commitDeltaVersionLogToIcebergTransaction( @@ -195,7 +188,7 @@ private void commitDeltaVersionLogToIcebergTransaction( // 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) + // 1. AppendFiles - 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) @@ -221,18 +214,18 @@ private void commitDeltaVersionLogToIcebergTransaction( } if (filesToAdd.size() > 0 && filesToRemove.size() > 0) { - // Overwrite_Files case + // OverwriteFiles case OverwriteFiles overwriteFiles = transaction.newOverwrite(); filesToAdd.forEach(overwriteFiles::addFile); filesToRemove.forEach(overwriteFiles::deleteFile); overwriteFiles.commit(); } else if (filesToAdd.size() > 0) { - // Append_Files case + // AppendFiles case AppendFiles appendFiles = transaction.newAppend(); filesToAdd.forEach(appendFiles::appendFile); appendFiles.commit(); } else if (filesToRemove.size() > 0) { - // Delete_Files case + // DeleteFiles case DeleteFiles deleteFiles = transaction.newDelete(); filesToRemove.forEach(deleteFiles::deleteFile); deleteFiles.commit(); @@ -253,27 +246,20 @@ public DataFile buildDataFileFromAction(Action action, Table table) { path = removeFile.getPath(); partitionValues = removeFile.getPartitionValues(); } else { - throw new IllegalStateException( - String.format( - "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName())); + throw new ValidationException( + "Unexpected action type for Delta Lake: %s", action.getClass().getSimpleName()); } String fullFilePath = getFullFilePath(path, deltaLog.getPath().toString()); if (partitionValues == null) { // For unpartitioned table, the partitionValues should be an empty map rather than null - throw new ValidationException("File %s does not specify a partitionValues", fullFilePath); + throw new IllegalArgumentException( + String.format("File %s does not specify a partitionValues", fullFilePath)); } FileFormat format = determineFileFormatFromPath(fullFilePath); - FileIO io = table.io(); - InputFile file; - if (io != null) { - file = io.newInputFile(fullFilePath); - } else { - file = HadoopInputFile.fromPath(new Path(fullFilePath), this.hadoopConfiguration); - } - + InputFile file = hadoopFileIO.newInputFile(fullFilePath); MetricsConfig metricsConfig = MetricsConfig.forTable(table); String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping nameMapping = @@ -322,19 +308,14 @@ private Metrics getMetricsForFile( } } - private static Map destTableProperties( - io.delta.standalone.Snapshot deltaSnapshot, - String originalLocation, - Map additionalProperties) { - Map properties = Maps.newHashMap(); - - properties.putAll(deltaSnapshot.getMetadata().getConfiguration()); - properties.putAll( + private Map destTableProperties( + io.delta.standalone.Snapshot deltaSnapshot, String originalLocation) { + additionalPropertiesBuilder.putAll(deltaSnapshot.getMetadata().getConfiguration()); + additionalPropertiesBuilder.putAll( ImmutableMap.of( MIGRATION_SOURCE_PROP, DELTA_SOURCE_VALUE, ORIGINAL_LOCATION_PROP, originalLocation)); - properties.putAll(additionalProperties); - return properties; + return additionalPropertiesBuilder.build(); } /** @@ -347,7 +328,8 @@ private static Map destTableProperties( */ @VisibleForTesting static String getFullFilePath(String path, String tableRoot) { - if (path.startsWith(tableRoot)) { + URI dataFileUri = URI.create(path); + if (dataFileUri.isAbsolute()) { return path; } else { return tableRoot + File.separator + path; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java similarity index 96% rename from delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java index 412c6483a330..8af654a97b38 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeDataTypeVisitor.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeDataTypeVisitor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.utils; +package org.apache.iceberg.delta; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.DataType; @@ -26,7 +26,7 @@ import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public abstract class DeltaLakeDataTypeVisitor { +abstract class DeltaLakeDataTypeVisitor { public static T visit(DataType type, DeltaLakeDataTypeVisitor visitor) { if (type instanceof StructType) { StructField[] fields = ((StructType) type).getFields(); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java similarity index 96% rename from delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java index ef964070e252..eeea53d3907c 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/utils/DeltaLakeTypeToType.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.utils; +package org.apache.iceberg.delta; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; @@ -40,7 +40,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -public class DeltaLakeTypeToType extends DeltaLakeDataTypeVisitor { +class DeltaLakeTypeToType extends DeltaLakeDataTypeVisitor { private final StructType root; private int nextId = 0; @@ -48,7 +48,7 @@ public class DeltaLakeTypeToType extends DeltaLakeDataTypeVisitor { this.root = null; } - public DeltaLakeTypeToType(StructType root) { + DeltaLakeTypeToType(StructType root) { this.root = root; this.nextId = root.getFields().length; } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java similarity index 99% rename from delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java rename to delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index f215badb0a22..44f270cd911f 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/utils/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.delta.utils; +package org.apache.iceberg.delta; import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index fe67da5b2966..3c4af5304b39 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -33,7 +33,7 @@ *

This class is the primary API for interacting with actions in Spark that users should use to * instantiate particular actions. */ -public class SparkActions implements SupportMigrationFromDeltaLake, ActionsProvider { +public class SparkActions implements ActionsProvider, SupportMigrationFromDeltaLake { private final SparkSession spark; From f4589e806b57e6745e93e6f24688415c7f06f196 Mon Sep 17 00:00:00 2001 From: JonasJ-ap Date: Fri, 6 Jan 2023 17:18:40 -0500 Subject: [PATCH 26/54] optimize the constructor to be more clean --- .../iceberg/delta/BaseMigrateDeltaLakeTableAction.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java index b0ba78c9f305..26b47c3aa234 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java @@ -92,12 +92,7 @@ public BaseMigrateDeltaLakeTableAction( String deltaTableLocation, TableIdentifier newTableIdentifier, Configuration hadoopConfiguration) { - this.icebergCatalog = icebergCatalog; - this.deltaTableLocation = deltaTableLocation; - this.newTableIdentifier = newTableIdentifier; - this.newTableLocation = deltaTableLocation; - this.deltaLog = DeltaLog.forTable(hadoopConfiguration, deltaTableLocation); - this.hadoopFileIO = new HadoopFileIO(hadoopConfiguration); + this(icebergCatalog, deltaTableLocation, newTableIdentifier, null, hadoopConfiguration); } public BaseMigrateDeltaLakeTableAction( From 59c96cb9f4633c8f1390bc63904954fe41d38af6 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Fri, 6 Jan 2023 23:13:06 -0500 Subject: [PATCH 27/54] move everthing to iceberg-delta-lake, build demo integration test (#35) * refactor mixin order * fix nit problems * add null check to the constructor * let copyFromDeltaLakeToIceberg return the number of totalDataFiles directly * use ImmutableMap.Builder * fix the problem in getFullFilePath * use hadoopFileIO to read dataFile * make type conversion util package-private * fix format issue * move everything to iceberg-delta-lake. Remove all changes made to iceberg-spark * fix test delta_core dependency * fix format * conditionally build the test * refactor to integrationTest * suppress warnings * test delta core 2.2.0 --- build.gradle | 45 +++++++++++ .../apache/iceberg/delta/SimpleRecord.java | 78 ++++++++++++++++++ .../iceberg/delta/SparkCatalogConfig.java | 69 ++++++++++++++++ .../SparkDeltaLakeMigrationTestBase.java | 9 ++- .../delta}/TestMigrateDeltaLakeTable.java | 81 ++++++++++++++++--- spark/v3.3/build.gradle | 6 -- .../MigrateDeltaLakeTableSparkAction.java | 57 ------------- .../iceberg/spark/actions/SparkActions.java | 33 +------- versions.props | 2 +- 9 files changed, 268 insertions(+), 112 deletions(-) create mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java create mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java rename {spark/v3.3/spark/src/test/java/org/apache/iceberg/spark => delta-lake/src/integration/java/org/apache/iceberg/delta}/SparkDeltaLakeMigrationTestBase.java (96%) rename {spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions => delta-lake/src/integration/java/org/apache/iceberg/delta}/TestMigrateDeltaLakeTable.java (73%) delete mode 100644 spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java diff --git a/build.gradle b/build.gradle index 841f562e7217..d217655e4408 100644 --- a/build.gradle +++ b/build.gradle @@ -49,6 +49,8 @@ plugins { } String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") +String sparkVersionsString = System.getProperty("sparkVersions") != null ? System.getProperty("sparkVersions") : System.getProperty("defaultSparkVersions") +List sparkVersions = sparkVersionsString != null && !sparkVersionsString.isEmpty() ? sparkVersionsString.split(",") : [] try { // apply these plugins in a try-catch block so that we can handle cases without .git directory @@ -441,6 +443,11 @@ project(':iceberg-aws') { } project(':iceberg-delta-lake') { + configurations { + integrationImplementation.extendsFrom testImplementation + integrationRuntime.extendsFrom testRuntimeOnly + } + dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') @@ -458,6 +465,44 @@ project(':iceberg-delta-lake') { exclude group: 'javax.servlet', module: 'servlet-api' exclude group: 'com.google.code.gson', module: 'gson' } + + if (sparkVersions.contains("3.3")) { + integrationImplementation "io.delta:delta-core_${scalaVersion}" + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.3_${scalaVersion}") + integrationImplementation("org.apache.hadoop:hadoop-minicluster") { + exclude group: 'org.apache.avro', module: 'avro' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + } + integrationImplementation project(path: ':iceberg-hive-metastore') + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:3.3.1") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.apache.arrow' + exclude group: 'org.apache.parquet' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + exclude group: 'org.roaringbitmap' + } + } + } + + if (sparkVersions.contains("3.3")) { + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + } + } + + task integrationTest(type: Test) { + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + } } } diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java new file mode 100644 index 000000000000..7a35bd581060 --- /dev/null +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +public class SimpleRecord { + private Integer id; + private String data; + + public SimpleRecord() {} + + public SimpleRecord(Integer id, String data) { + this.id = id; + this.data = data; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public String getData() { + return data; + } + + public void setData(String data) { + this.data = data; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SimpleRecord record = (SimpleRecord) o; + return Objects.equal(id, record.id) && Objects.equal(data, record.data); + } + + @Override + public int hashCode() { + return Objects.hashCode(id, data); + } + + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("{\"id\"="); + buffer.append(id); + buffer.append(",\"data\"=\""); + buffer.append(data); + buffer.append("\"}"); + return buffer.toString(); + } +} diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java new file mode 100644 index 000000000000..40631eae2c2b --- /dev/null +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; + +public enum SparkCatalogConfig { + HIVE( + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default")), + HADOOP("testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop")), + SPARK( + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + )); + + private final String catalogName; + private final String implementation; + private final Map properties; + + SparkCatalogConfig(String catalogName, String implementation, Map properties) { + this.catalogName = catalogName; + this.implementation = implementation; + this.properties = properties; + } + + public String catalogName() { + return catalogName; + } + + public String implementation() { + return implementation; + } + + public Map properties() { + return properties; + } +} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java similarity index 96% rename from spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java rename to delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java index 8ea9c67f7851..a2ce0a634fec 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkDeltaLakeMigrationTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java @@ -16,9 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.spark; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; +package org.apache.iceberg.delta; import java.io.File; import java.io.IOException; @@ -49,6 +47,7 @@ import org.junit.Rule; import org.junit.rules.TemporaryFolder; +@SuppressWarnings("VisibilityModifier") public abstract class SparkDeltaLakeMigrationTestBase { protected static final Object ANY = new Object(); @@ -69,7 +68,9 @@ public static void startMetastoreAndSpark() { SparkSession.builder() .master("local[2]") .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") - .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config( + "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname, + hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") // Needed for Delta Lake tests .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java similarity index 73% rename from spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java rename to delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java index 71b11c9a37c7..448715846c5b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java @@ -16,23 +16,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.spark.actions; +package org.apache.iceberg.delta; import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.stream.IntStream; -import org.apache.iceberg.delta.MigrateDeltaLakeTable; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkDeltaLakeMigrationTestBase; +import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.CatalogExtension; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.delta.catalog.DeltaCatalog; import org.junit.After; @@ -59,11 +60,14 @@ public static Object[][] parameters() { "delta", DeltaCatalog.class.getName(), ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "parquet-enabled", "true", + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync + "false" // Spark will delete tables using v1, leaving the cache out of sync ) } }; @@ -166,7 +170,7 @@ public void testMigratePartitioned() { catalogName = defaultSparkCatalog; String newTableIdentifier = destName("iceberg_table"); MigrateDeltaLakeTable.Result result = - SparkActions.get().migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); + migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); // Compare the results List oldResults = spark.sql("SELECT * FROM " + partitionedIdentifier).collectAsList(); @@ -187,9 +191,7 @@ public void testMigrateUnpartitioned() { catalogName = defaultSparkCatalog; String newTableIdentifier = destName("iceberg_table_unpartitioned"); MigrateDeltaLakeTable.Result result = - SparkActions.get() - .migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation) - .execute(); + migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation).execute(); // Compare the results List oldResults = spark.sql("SELECT * FROM " + unpartitionedIdentifier).collectAsList(); @@ -207,4 +209,59 @@ private String destName(String dest) { return catalogName + "." + NAMESPACE + "." + catalogName + "_" + type + "_" + dest; } } + + private MigrateDeltaLakeTable migrateDeltaLakeTable( + String newTableIdentifier, String deltaTableLocation) { + String ctx = "delta lake migrate target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + Spark3Util.CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeTableSparkAction( + spark, + deltaTableLocation, + catalogAndIdent.identifier().toString(), + catalogAndIdent.catalog().name()); + } + + public MigrateDeltaLakeTable migrateDeltaLakeTable( + String newTableIdentifier, String deltaTableLocation, String newTableLocation) { + String ctx = "delta lake migrate target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + Spark3Util.CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new MigrateDeltaLakeTableSparkAction( + spark, + deltaTableLocation, + catalogAndIdent.identifier().toString(), + catalogAndIdent.catalog().name(), + newTableLocation); + } + + private class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableAction { + MigrateDeltaLakeTableSparkAction( + SparkSession spark, + String deltaTableLocation, + String newTableIdentifier, + String catalogName) { + super( + Spark3Util.loadIcebergCatalog(spark, catalogName), + deltaTableLocation, + TableIdentifier.parse(newTableIdentifier), + spark.sessionState().newHadoopConf()); + } + + MigrateDeltaLakeTableSparkAction( + SparkSession spark, + String deltaTableLocation, + String newTableIdentifier, + String catalogName, + String newTableLocation) { + super( + Spark3Util.loadIcebergCatalog(spark, catalogName), + deltaTableLocation, + TableIdentifier.parse(newTableIdentifier), + newTableLocation, + spark.sessionState().newHadoopConf()); + } + } } diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 13061b9015e7..6e97ed147208 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -59,7 +59,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-orc') implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') - implementation project(':iceberg-delta-lake') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}") compileOnly "com.google.errorprone:error_prone_annotations" @@ -91,11 +90,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'com.google.code.findbugs', module: 'jsr305' } - compileOnly "io.delta:delta-standalone_${scalaVersion}" - - // Needed to write Delta Lake tables for testing - testImplementation "io.delta:delta-core_${scalaVersion}" - testImplementation("org.apache.hadoop:hadoop-minicluster") { exclude group: 'org.apache.avro', module: 'avro' // to make sure netty libs only come from project(':iceberg-arrow') diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java deleted file mode 100644 index e9c7fc3f3af7..000000000000 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateDeltaLakeTableSparkAction.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.delta.BaseMigrateDeltaLakeTableAction; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.spark.sql.SparkSession; - -/** - * 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 extends BaseMigrateDeltaLakeTableAction { - MigrateDeltaLakeTableSparkAction( - SparkSession spark, - String deltaTableLocation, - String newTableIdentifier, - String catalogName) { - super( - Spark3Util.loadIcebergCatalog(spark, catalogName), - deltaTableLocation, - TableIdentifier.parse(newTableIdentifier), - spark.sessionState().newHadoopConf()); - } - - MigrateDeltaLakeTableSparkAction( - SparkSession spark, - String deltaTableLocation, - String newTableIdentifier, - String catalogName, - String newTableLocation) { - super( - Spark3Util.loadIcebergCatalog(spark, catalogName), - deltaTableLocation, - TableIdentifier.parse(newTableIdentifier), - newTableLocation, - spark.sessionState().newHadoopConf()); - } -} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 3c4af5304b39..8c886adf510e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,8 +20,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; -import org.apache.iceberg.delta.MigrateDeltaLakeTable; -import org.apache.iceberg.delta.SupportMigrationFromDeltaLake; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -33,7 +31,7 @@ *

This class is the primary API for interacting with actions in Spark that users should use to * instantiate particular actions. */ -public class SparkActions implements ActionsProvider, SupportMigrationFromDeltaLake { +public class SparkActions implements ActionsProvider { private final SparkSession spark; @@ -69,35 +67,6 @@ public MigrateTableSparkAction migrateTable(String tableIdent) { spark, catalogAndIdent.catalog(), catalogAndIdent.identifier()); } - @Override - public MigrateDeltaLakeTable migrateDeltaLakeTable( - String newTableIdentifier, String deltaTableLocation) { - String ctx = "delta lake migrate target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeTableSparkAction( - spark, - deltaTableLocation, - catalogAndIdent.identifier().toString(), - catalogAndIdent.catalog().name()); - } - - @Override - public MigrateDeltaLakeTable migrateDeltaLakeTable( - String newTableIdentifier, String deltaTableLocation, String newTableLocation) { - String ctx = "delta lake migrate target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeTableSparkAction( - spark, - deltaTableLocation, - catalogAndIdent.identifier().toString(), - catalogAndIdent.catalog().name(), - newTableLocation); - } - @Override public RewriteDataFilesSparkAction rewriteDataFiles(Table table) { return new RewriteDataFilesSparkAction(spark, table); diff --git a/versions.props b/versions.props index b357f260b898..fcd540fd1db3 100644 --- a/versions.props +++ b/versions.props @@ -45,4 +45,4 @@ org.mock-server:mockserver-netty = 5.13.2 org.mock-server:mockserver-client-java = 5.13.2 com.esotericsoftware:kryo = 4.0.2 org.eclipse.jetty:* = 9.4.43.v20210629 -io.delta:delta-core_* = 2.1.0 +io.delta:delta-core_* = 2.2.0 From afd783b02ef2fba08940794bcecb16a94a425e72 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Mon, 9 Jan 2023 01:12:35 -0500 Subject: [PATCH 28/54] optimize api structure, refactor the integration test, add more tests(#39) * use validation exception for unsupported types * check result file count in integration test * fix format * add tableLocation api and remove constructors * add javadoc for constructor * remove unnecessary test * use UUID to generate table records * resolve format issue * rename everything from migrate to snapshot * simplify test configuration round 1 * refactor test spark integration * refactor correctness check to helper function * add test for table location and table properties --- build.gradle | 6 + .../SnapshotDeltaLakeSparkIntegration.java | 42 +++ ...va => SparkDeltaLakeSnapshotTestBase.java} | 26 +- .../delta/TestMigrateDeltaLakeTable.java | 267 ---------------- .../delta/TestSnapshotDeltaLakeTable.java | 285 ++++++++++++++++++ ... => BaseSnapshotDeltaLakeTableAction.java} | 63 ++-- ...seSnapshotDeltaLakeTableActionResult.java} | 12 +- .../iceberg/delta/DeltaLakeTypeToType.java | 4 +- ...Table.java => SnapshotDeltaLakeTable.java} | 21 +- ...java => SupportSnapshotFromDeltaLake.java} | 8 +- .../delta/TestBaseMigrateDeltaLake.java | 42 --- 11 files changed, 407 insertions(+), 369 deletions(-) create mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java rename delta-lake/src/integration/java/org/apache/iceberg/delta/{SparkDeltaLakeMigrationTestBase.java => SparkDeltaLakeSnapshotTestBase.java} (89%) delete mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java create mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java rename delta-lake/src/main/java/org/apache/iceberg/delta/{BaseMigrateDeltaLakeTableAction.java => BaseSnapshotDeltaLakeTableAction.java} (86%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{BaseMigrateDeltaLakeTableActionResult.java => BaseSnapshotDeltaLakeTableActionResult.java} (71%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{MigrateDeltaLakeTable.java => SnapshotDeltaLakeTable.java} (70%) rename delta-lake/src/main/java/org/apache/iceberg/delta/{SupportMigrationFromDeltaLake.java => SupportSnapshotFromDeltaLake.java} (85%) delete mode 100644 delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java diff --git a/build.gradle b/build.gradle index d217655e4408..d6cf4d88745d 100644 --- a/build.gradle +++ b/build.gradle @@ -443,6 +443,8 @@ project(':iceberg-aws') { } project(':iceberg-delta-lake') { + // use integration test since we can take advantages of spark 3.3 to read datafiles of delta lake table + // and create some tests involving sql query. configurations { integrationImplementation.extendsFrom testImplementation integrationRuntime.extendsFrom testRuntimeOnly @@ -466,6 +468,8 @@ project(':iceberg-delta-lake') { exclude group: 'com.google.code.gson', module: 'gson' } + // The newest version of delta-core uses Spark 3.3.*. Since its only for test, we do + // not need to include older version of delta-core if (sparkVersions.contains("3.3")) { integrationImplementation "io.delta:delta-core_${scalaVersion}" integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.3_${scalaVersion}") @@ -489,6 +493,8 @@ project(':iceberg-delta-lake') { } } + // The newest version of delta-core uses Spark 3.3.*. The integration test should only be built + // if iceberg-spark-3.3 is available if (sparkVersions.contains("3.3")) { sourceSets { integration { diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java new file mode 100644 index 000000000000..90c4867979db --- /dev/null +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; + +class SnapshotDeltaLakeSparkIntegration { + + private SnapshotDeltaLakeSparkIntegration() {} + + static SnapshotDeltaLakeTable snapshotDeltaLakeTable( + SparkSession spark, String newTableIdentifier, String deltaTableLocation) { + String ctx = "delta lake snapshot target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + Spark3Util.CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new BaseSnapshotDeltaLakeTableAction( + Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name()), + deltaTableLocation, + TableIdentifier.parse(catalogAndIdent.identifier().toString()), + spark.sessionState().newHadoopConf()); + } +} diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java similarity index 89% rename from delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java rename to delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java index a2ce0a634fec..59bc7626dff4 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeMigrationTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java @@ -48,9 +48,7 @@ import org.junit.rules.TemporaryFolder; @SuppressWarnings("VisibilityModifier") -public abstract class SparkDeltaLakeMigrationTestBase { - protected static final Object ANY = new Object(); - +public abstract class SparkDeltaLakeSnapshotTestBase { protected static TestHiveMetastore metastore = null; protected static HiveConf hiveConf = null; protected static SparkSession spark = null; @@ -60,11 +58,11 @@ public abstract class SparkDeltaLakeMigrationTestBase { @BeforeClass public static void startMetastoreAndSpark() { - SparkDeltaLakeMigrationTestBase.metastore = new TestHiveMetastore(); + SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore(); metastore.start(); - SparkDeltaLakeMigrationTestBase.hiveConf = metastore.hiveConf(); + SparkDeltaLakeSnapshotTestBase.hiveConf = metastore.hiveConf(); - SparkDeltaLakeMigrationTestBase.spark = + SparkDeltaLakeSnapshotTestBase.spark = SparkSession.builder() .master("local[2]") .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") @@ -77,7 +75,7 @@ public static void startMetastoreAndSpark() { .enableHiveSupport() .getOrCreate(); - SparkDeltaLakeMigrationTestBase.catalog = + SparkDeltaLakeSnapshotTestBase.catalog = (HiveCatalog) CatalogUtil.loadCatalog( HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); @@ -91,16 +89,16 @@ public static void startMetastoreAndSpark() { @AfterClass public static void stopMetastoreAndSpark() throws Exception { - SparkDeltaLakeMigrationTestBase.catalog = null; + SparkDeltaLakeSnapshotTestBase.catalog = null; metastore.stop(); - SparkDeltaLakeMigrationTestBase.metastore = null; + SparkDeltaLakeSnapshotTestBase.metastore = null; spark.stop(); - SparkDeltaLakeMigrationTestBase.spark = null; + SparkDeltaLakeSnapshotTestBase.spark = null; } @BeforeClass public static void createWarehouse() throws IOException { - SparkDeltaLakeMigrationTestBase.warehouse = File.createTempFile("warehouse", null); + SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null); Assert.assertTrue(warehouse.delete()); } @@ -121,15 +119,15 @@ public static void dropWarehouse() throws IOException { protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); protected final String tableName; - public SparkDeltaLakeMigrationTestBase() { + public SparkDeltaLakeSnapshotTestBase() { this(SparkCatalogConfig.HADOOP); } - public SparkDeltaLakeMigrationTestBase(SparkCatalogConfig config) { + public SparkDeltaLakeSnapshotTestBase(SparkCatalogConfig config) { this(config.catalogName(), config.implementation(), config.properties()); } - public SparkDeltaLakeMigrationTestBase( + public SparkDeltaLakeSnapshotTestBase( String catalogName, String implementation, Map config) { this.catalogName = catalogName; this.validationCatalog = diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java deleted file mode 100644 index 448715846c5b..000000000000 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestMigrateDeltaLakeTable.java +++ /dev/null @@ -1,267 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.delta; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.stream.IntStream; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.catalog.CatalogExtension; -import org.apache.spark.sql.connector.catalog.CatalogPlugin; -import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.delta.catalog.DeltaCatalog; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMigrateDeltaLakeTable extends SparkDeltaLakeMigrationTestBase { - private static final String NAMESPACE = "default"; - - private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; - private String partitionedIdentifier; - private String unpartitionedIdentifier; - - @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] { - "delta", - DeltaCatalog.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 temp = new TemporaryFolder(); - @Rule public TemporaryFolder other = new TemporaryFolder(); - - private final String partitionedTableName = "partitioned_table"; - private final String unpartitionedTableName = "unpartitioned_table"; - - private final String defaultSparkCatalog = "spark_catalog"; - private String partitionedLocation; - private String unpartitionedLocation; - private final String type; - private TableCatalog catalog; - - private String catalogName; - - public TestMigrateDeltaLakeTable( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - spark - .conf() - .set("spark.sql.catalog." + defaultSparkCatalog, SparkSessionCatalog.class.getName()); - this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); - this.type = config.get("type"); - this.catalogName = catalogName; - } - - @Before - public void before() { - try { - File partitionedFolder = temp.newFolder(); - File unpartitionedFolder = other.newFolder(); - partitionedLocation = partitionedFolder.toURI().toString(); - unpartitionedLocation = unpartitionedFolder.toURI().toString(); - } catch (IOException e) { - throw new RuntimeException(e); - } - - partitionedIdentifier = destName(partitionedTableName); - unpartitionedIdentifier = destName(unpartitionedTableName); - - CatalogExtension delta = - (CatalogExtension) spark.sessionState().catalogManager().catalog("delta"); - // This needs to be set, otherwise Delta operations fail as the catalog is designed to override - // the default catalog (spark_catalog). - delta.setDelegateCatalog(spark.sessionState().catalogManager().currentCatalog()); - - spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); - spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); - - // Create a partitioned and unpartitioned table, doing a few inserts on each - IntStream.range(0, 3) - .forEach( - i -> { - List record = - Lists.newArrayList(new SimpleRecord(i, ALPHABET.substring(i, i + 1))); - - Dataset df = spark.createDataFrame(record, SimpleRecord.class); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .partitionBy("id") - .option("path", partitionedLocation) - .saveAsTable(partitionedIdentifier); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .option("path", unpartitionedLocation) - .saveAsTable(unpartitionedIdentifier); - }); - - // Delete a record from the table - spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); - spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); - - // 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() throws IOException { - // Drop the hive table. - spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(partitionedTableName))); - spark.sql(String.format("DROP TABLE IF EXISTS %s", destName(unpartitionedTableName))); - } - - @Test - public void testMigratePartitioned() { - // This will test the scenario that the user switches the configuration and sets the default - // catalog to be Iceberg - // AFTER they had made it Delta and written a delta table there - spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); - - catalogName = defaultSparkCatalog; - String newTableIdentifier = destName("iceberg_table"); - MigrateDeltaLakeTable.Result result = - migrateDeltaLakeTable(newTableIdentifier, partitionedLocation).execute(); - - // Compare the results - List oldResults = spark.sql("SELECT * FROM " + partitionedIdentifier).collectAsList(); - List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); - - Assert.assertEquals(oldResults.size(), newResults.size()); - Assert.assertTrue(newResults.containsAll(oldResults)); - Assert.assertTrue(oldResults.containsAll(newResults)); - } - - @Test - public void testMigrateUnpartitioned() { - // This will test the scenario that the user switches the configuration and sets the default - // catalog to be Iceberg - // AFTER they had made it Delta and written a delta table there - spark.sessionState().catalogManager().setCurrentCatalog(defaultSparkCatalog); - - catalogName = defaultSparkCatalog; - String newTableIdentifier = destName("iceberg_table_unpartitioned"); - MigrateDeltaLakeTable.Result result = - migrateDeltaLakeTable(newTableIdentifier, unpartitionedLocation).execute(); - - // Compare the results - List oldResults = spark.sql("SELECT * FROM " + unpartitionedIdentifier).collectAsList(); - List newResults = spark.sql("SELECT * FROM " + newTableIdentifier).collectAsList(); - - Assert.assertEquals(oldResults.size(), newResults.size()); - Assert.assertTrue(newResults.containsAll(oldResults)); - Assert.assertTrue(oldResults.containsAll(newResults)); - } - - private String destName(String dest) { - if (catalogName.equals("spark_catalog")) { - return NAMESPACE + "." + catalogName + "_" + type + "_" + dest; - } else { - return catalogName + "." + NAMESPACE + "." + catalogName + "_" + type + "_" + dest; - } - } - - private MigrateDeltaLakeTable migrateDeltaLakeTable( - String newTableIdentifier, String deltaTableLocation) { - String ctx = "delta lake migrate target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - Spark3Util.CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeTableSparkAction( - spark, - deltaTableLocation, - catalogAndIdent.identifier().toString(), - catalogAndIdent.catalog().name()); - } - - public MigrateDeltaLakeTable migrateDeltaLakeTable( - String newTableIdentifier, String deltaTableLocation, String newTableLocation) { - String ctx = "delta lake migrate target"; - CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); - Spark3Util.CatalogAndIdentifier catalogAndIdent = - Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new MigrateDeltaLakeTableSparkAction( - spark, - deltaTableLocation, - catalogAndIdent.identifier().toString(), - catalogAndIdent.catalog().name(), - newTableLocation); - } - - private class MigrateDeltaLakeTableSparkAction extends BaseMigrateDeltaLakeTableAction { - MigrateDeltaLakeTableSparkAction( - SparkSession spark, - String deltaTableLocation, - String newTableIdentifier, - String catalogName) { - super( - Spark3Util.loadIcebergCatalog(spark, catalogName), - deltaTableLocation, - TableIdentifier.parse(newTableIdentifier), - spark.sessionState().newHadoopConf()); - } - - MigrateDeltaLakeTableSparkAction( - SparkSession spark, - String deltaTableLocation, - String newTableIdentifier, - String catalogName, - String newTableLocation) { - super( - Spark3Util.loadIcebergCatalog(spark, catalogName), - deltaTableLocation, - TableIdentifier.parse(newTableIdentifier), - newTableLocation, - spark.sessionState().newHadoopConf()); - } - } -} diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java new file mode 100644 index 000000000000..938e239e60dc --- /dev/null +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -0,0 +1,285 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.IntStream; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.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.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { + private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; + private static final String DELTA_SOURCE_VALUE = "delta"; + private static final String ORIGINAL_LOCATION_PROP = "original_location"; + private static final String NAMESPACE = "default"; + private String partitionedIdentifier; + private String unpartitionedIdentifier; + private static final String defaultSparkCatalog = "spark_catalog"; + private static final String icebergCatalogName = "iceberg_hive"; + + @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] { + icebergCatalogName, + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ) + } + }; + } + + @Rule public TemporaryFolder temp1 = new TemporaryFolder(); + @Rule public TemporaryFolder temp2 = new TemporaryFolder(); + @Rule public TemporaryFolder temp3 = new TemporaryFolder(); + + private final String partitionedTableName = "partitioned_table"; + private final String unpartitionedTableName = "unpartitioned_table"; + + private String partitionedLocation; + private String unpartitionedLocation; + private String newIcebergTableLocation; + + public TestSnapshotDeltaLakeTable( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); + } + + @Before + public void before() { + try { + File partitionedFolder = temp1.newFolder(); + File unpartitionedFolder = temp2.newFolder(); + File newIcebergTableFolder = temp3.newFolder(); + partitionedLocation = partitionedFolder.toURI().toString(); + unpartitionedLocation = unpartitionedFolder.toURI().toString(); + newIcebergTableLocation = newIcebergTableFolder.toURI().toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName); + unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName); + + spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); + + // Create a partitioned and unpartitioned table, doing a few inserts on each + IntStream.range(0, 3) + .forEach( + i -> { + List record = + Lists.newArrayList(new SimpleRecord(i, UUID.randomUUID().toString())); + + Dataset df = spark.createDataFrame(record, SimpleRecord.class); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .partitionBy("id") + .option("path", partitionedLocation) + .saveAsTable(partitionedIdentifier); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .option("path", unpartitionedLocation) + .saveAsTable(unpartitionedIdentifier); + }); + + // Delete a record from the table + spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); + spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); + + // 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 the hive table. + spark.sql( + String.format( + "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName))); + spark.sql( + String.format( + "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, unpartitionedTableName))); + } + + @Test + public void testBasicSnapshotPartitioned() { + String newTableIdentifier = destName(icebergCatalogName, "iceberg_table"); + SnapshotDeltaLakeTable.Result result = + SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, partitionedLocation) + .execute(); + + checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, partitionedLocation); + } + + @Test + public void testBasicSnapshotUnpartitioned() { + String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned"); + SnapshotDeltaLakeTable.Result result = + SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, unpartitionedLocation) + .execute(); + + checkSnapshotIntegrity( + unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); + } + + @Test + public void testSnapshotWithNewLocation() { + String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location"); + SnapshotDeltaLakeTable.Result result = + SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, partitionedLocation) + .tableLocation(newIcebergTableLocation) + .execute(); + + checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); + } + + @Test + public void testSnapshotWithAdditionalProperties() { + String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties"); + SnapshotDeltaLakeTable.Result result = + SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, unpartitionedLocation) + .tableProperty("test1", "test1") + .tableProperties(ImmutableMap.of("test2", "test2", "test3", "test3", "test4", "test4")) + .execute(); + + checkSnapshotIntegrity( + unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); + checkIcebergTableProperties( + newTableIdentifier, + ImmutableMap.of("test1", "test1", "test2", "test2", "test3", "test3", "test4", "test4"), + unpartitionedLocation); + } + + private void checkSnapshotIntegrity( + String deltaTableLocation, + String deltaTableIdentifier, + String icebergTableIdentifier, + SnapshotDeltaLakeTable.Result snapshotReport) { + DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation); + + List deltaTableContents = + spark.sql("SELECT * FROM " + deltaTableIdentifier).collectAsList(); + List icebergTableContents = + spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList(); + + Assert.assertEquals( + "The original table and the transformed one should have the same size", + deltaTableContents.size(), + icebergTableContents.size()); + Assert.assertTrue( + "The original table and the transformed one should have the same contents", + icebergTableContents.containsAll(deltaTableContents)); + Assert.assertTrue( + "The original table and the transformed one should have the same contents", + deltaTableContents.containsAll(icebergTableContents)); + Assert.assertEquals( + "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table", + deltaLog.update().getAllFiles().size(), + snapshotReport.snapshotDataFilesCount()); + } + + private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) { + Table icebergTable = getIcebergTable(icebergTableIdentifier); + Assert.assertEquals( + "The iceberg table should have the expected location", + expectedLocation, + icebergTable.location()); + } + + private void checkIcebergTableProperties( + String icebergTableIdentifier, + Map expectedAdditionalProperties, + String deltaTableLocation) { + Table icebergTable = getIcebergTable(icebergTableIdentifier); + ImmutableMap.Builder expectedPropertiesBuilder = ImmutableMap.builder(); + // The snapshot action will put some fixed properties to the table + expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE); + expectedPropertiesBuilder.putAll(expectedAdditionalProperties); + ImmutableMap expectedProperties = expectedPropertiesBuilder.build(); + Assert.assertTrue( + "The snapshot iceberg table should have the expected properties, both default and user added ones", + icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet())); + Assert.assertTrue( + "The snapshot iceberg table's property should contains the original location", + icebergTable.properties().containsKey(ORIGINAL_LOCATION_PROP) + && icebergTable.properties().get(ORIGINAL_LOCATION_PROP).equals(deltaTableLocation)); + } + + private Table getIcebergTable(String icebergTableIdentifier) { + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + Spark3Util.CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier( + "test catalog", spark, icebergTableIdentifier, defaultCatalog); + return Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name()) + .loadTable(TableIdentifier.parse(catalogAndIdent.identifier().toString())); + } + + private String destName(String catalogName, String dest) { + if (catalogName.equals(defaultSparkCatalog)) { + return NAMESPACE + "." + catalogName + "_" + dest; + } + return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest; + } +} diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java similarity index 86% rename from delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 26b47c3aa234..f0bb1652a80d 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -55,7 +55,6 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.OrcMetrics; import org.apache.iceberg.parquet.ParquetUtil; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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; @@ -64,15 +63,15 @@ import org.slf4j.LoggerFactory; /** - * Takes a Delta Lake table's location and attempts to transform it into an Iceberg table in an + * Takes a Delta Lake table's location and attempts to create an Iceberg table snapshot in an * optional user-specified location (default to the Delta Lake table's location) with a different * identifier. */ -public class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { +public class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { - private static final Logger LOG = LoggerFactory.getLogger(BaseMigrateDeltaLakeTableAction.class); + private static final Logger LOG = LoggerFactory.getLogger(BaseSnapshotDeltaLakeTableAction.class); - private static final String MIGRATION_SOURCE_PROP = "migration_source"; + 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 PARQUET_SUFFIX = ".parquet"; @@ -84,43 +83,52 @@ public class BaseMigrateDeltaLakeTableAction implements MigrateDeltaLakeTable { private final Catalog icebergCatalog; private final String deltaTableLocation; private final TableIdentifier newTableIdentifier; - private final String newTableLocation; - private final HadoopFileIO hadoopFileIO; + private String newTableLocation; + private final HadoopFileIO deltaLakeFileIO; - public BaseMigrateDeltaLakeTableAction( - Catalog icebergCatalog, - String deltaTableLocation, - TableIdentifier newTableIdentifier, - Configuration hadoopConfiguration) { - this(icebergCatalog, deltaTableLocation, newTableIdentifier, null, hadoopConfiguration); - } - - public BaseMigrateDeltaLakeTableAction( + /** + * Snapshot a delta lake table to be an iceberg table. The action will read the delta lake table's + * log through the table's path, create a new iceberg table using the given icebergCatalog and + * newTableIdentifier, and commit all changes in one iceberg transaction. + * + *

The new table will only be created if the snapshot is successful. + * + * @param icebergCatalog the iceberg catalog to create the iceberg table + * @param deltaTableLocation the delta lake table's path + * @param newTableIdentifier the identifier of the new iceberg table + * @param deltaLakeConfiguration the hadoop configuration to access the delta lake table + */ + public BaseSnapshotDeltaLakeTableAction( Catalog icebergCatalog, String deltaTableLocation, TableIdentifier newTableIdentifier, - String newTableLocation, - Configuration hadoopConfiguration) { + Configuration deltaLakeConfiguration) { this.icebergCatalog = icebergCatalog; this.deltaTableLocation = deltaTableLocation; this.newTableIdentifier = newTableIdentifier; - this.newTableLocation = newTableLocation == null ? deltaTableLocation : newTableLocation; - this.deltaLog = DeltaLog.forTable(hadoopConfiguration, deltaTableLocation); - this.hadoopFileIO = new HadoopFileIO(hadoopConfiguration); + this.newTableLocation = deltaTableLocation; + this.deltaLog = DeltaLog.forTable(deltaLakeConfiguration, deltaTableLocation); + this.deltaLakeFileIO = new HadoopFileIO(deltaLakeConfiguration); } @Override - public MigrateDeltaLakeTable tableProperties(Map properties) { + public SnapshotDeltaLakeTable tableProperties(Map properties) { additionalPropertiesBuilder.putAll(properties); return this; } @Override - public MigrateDeltaLakeTable tableProperty(String name, String value) { + public SnapshotDeltaLakeTable tableProperty(String name, String value) { additionalPropertiesBuilder.put(name, value); return this; } + @Override + public SnapshotDeltaLakeTable tableLocation(String location) { + this.newTableLocation = location; + return this; + } + @Override public Result execute() { io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); @@ -140,7 +148,7 @@ public Result execute() { "Successfully loaded Iceberg metadata for {} files in {}", totalDataFiles, deltaTableLocation); - return new BaseMigrateDeltaLakeTableActionResult(totalDataFiles); + return new BaseSnapshotDeltaLakeTableActionResult(totalDataFiles); } private Schema convertDeltaLakeSchema(io.delta.standalone.types.StructType deltaSchema) { @@ -254,7 +262,7 @@ public DataFile buildDataFileFromAction(Action action, Table table) { } FileFormat format = determineFileFormatFromPath(fullFilePath); - InputFile file = hadoopFileIO.newInputFile(fullFilePath); + InputFile file = deltaLakeFileIO.newInputFile(fullFilePath); MetricsConfig metricsConfig = MetricsConfig.forTable(table); String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping nameMapping = @@ -308,7 +316,7 @@ private Map destTableProperties( additionalPropertiesBuilder.putAll(deltaSnapshot.getMetadata().getConfiguration()); additionalPropertiesBuilder.putAll( ImmutableMap.of( - MIGRATION_SOURCE_PROP, DELTA_SOURCE_VALUE, ORIGINAL_LOCATION_PROP, originalLocation)); + SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE, ORIGINAL_LOCATION_PROP, originalLocation)); return additionalPropertiesBuilder.build(); } @@ -321,8 +329,7 @@ private Map destTableProperties( * (either absolute or relative) * @param tableRoot the root path of the delta table */ - @VisibleForTesting - static String getFullFilePath(String path, String tableRoot) { + private static String getFullFilePath(String path, String tableRoot) { URI dataFileUri = URI.create(path); if (dataFileUri.isAbsolute()) { return path; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java similarity index 71% rename from delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java index 5e094b913fd9..c618e5d6dc47 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseMigrateDeltaLakeTableActionResult.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java @@ -18,16 +18,16 @@ */ package org.apache.iceberg.delta; -public class BaseMigrateDeltaLakeTableActionResult implements MigrateDeltaLakeTable.Result { +public class BaseSnapshotDeltaLakeTableActionResult implements SnapshotDeltaLakeTable.Result { - private final long migratedDataFilesCount; + private final long snapshotDataFilesCount; - public BaseMigrateDeltaLakeTableActionResult(long migratedDataFilesCount) { - this.migratedDataFilesCount = migratedDataFilesCount; + public BaseSnapshotDeltaLakeTableActionResult(long snapshotDataFilesCount) { + this.snapshotDataFilesCount = snapshotDataFilesCount; } @Override - public long migratedDataFilesCount() { - return migratedDataFilesCount; + public long snapshotDataFilesCount() { + return snapshotDataFilesCount; } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java index eeea53d3907c..e0bb5a7a7342 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java @@ -36,6 +36,7 @@ import io.delta.standalone.types.StructType; import io.delta.standalone.types.TimestampType; import java.util.List; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -119,7 +120,6 @@ public Type map(MapType map, Type keyType, Type valueType) { @SuppressWarnings("checkstyle:CyclomaticComplexity") @Override public Type atomic(DataType atomic) { - // TODO: pay attention to Delta's NullType, currently unhandled if (atomic instanceof BooleanType) { return Types.BooleanType.get(); @@ -153,6 +153,6 @@ public Type atomic(DataType atomic) { return Types.BinaryType.get(); } - throw new UnsupportedOperationException("Not a supported type: " + atomic.getCatalogString()); + throw new ValidationException("Not a supported type: %s", atomic.getCatalogString()); } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java similarity index 70% rename from delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java index 709dd0e35060..6884da4f1680 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/MigrateDeltaLakeTable.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java @@ -21,9 +21,9 @@ import java.util.Map; import org.apache.iceberg.actions.Action; -/** Migrates an existing Delta Lake table to Iceberg in place. */ -public interface MigrateDeltaLakeTable - extends Action { +/** Snapshot an existing Delta Lake table to Iceberg in place. */ +public interface SnapshotDeltaLakeTable + extends Action { /** * Sets table properties in the newly created Iceberg table. Any properties with the same key name @@ -32,7 +32,7 @@ public interface MigrateDeltaLakeTable * @param properties a map of properties to set * @return this for method chaining */ - MigrateDeltaLakeTable tableProperties(Map properties); + SnapshotDeltaLakeTable tableProperties(Map properties); /** * Sets a table property in the newly created Iceberg table. Any properties with the same key will @@ -42,12 +42,21 @@ public interface MigrateDeltaLakeTable * @param value a table property value * @return this for method chaining */ - MigrateDeltaLakeTable tableProperty(String name, String value); + SnapshotDeltaLakeTable tableProperty(String name, String value); + + /** + * Sets the location of the newly created Iceberg table. Default location is the same as the Delta + * Lake table. + * + * @param location a path to the new table location + * @return this for method chaining + */ + SnapshotDeltaLakeTable tableLocation(String location); /** The action result that contains a summary of the execution. */ interface Result { /** Returns the number of migrated data files. */ - long migratedDataFilesCount(); + long snapshotDataFilesCount(); } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java similarity index 85% rename from delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java index b838d327445f..bb2eb4c0fb9c 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportMigrationFromDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java @@ -19,18 +19,18 @@ package org.apache.iceberg.delta; /** - * An API that should be implemented by query engine integrations that want to support migration + * An API that should be implemented by query engine integrations that want to support snapshotting * from Delta Lake table to Iceberg table. */ -public interface SupportMigrationFromDeltaLake { +public interface SupportSnapshotFromDeltaLake { /** Initiates an action to migrate an existing Delta Lake table to Iceberg. */ - default MigrateDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { + default SnapshotDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement migrateDeltaLakeTable"); } - default MigrateDeltaLakeTable migrateDeltaLakeTable( + default SnapshotDeltaLakeTable migrateDeltaLakeTable( String tableIdent, String deltaS3Location, String newTableLocation) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement migrateDeltaLakeTable"); diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java deleted file mode 100644 index 932b9a30c81f..000000000000 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseMigrateDeltaLake.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.delta; - -import java.io.File; -import org.junit.Assert; -import org.junit.Test; - -public class TestBaseMigrateDeltaLake { - - @Test - public void testGetFullFilePath() { - String fileName = "part-00000-12345678-1234-1234-1234-123456789012.parquet"; - String tableRoot = "s3://bucket/table"; - String relativeFilePath = "id=0" + File.separator + fileName; - String absoluteFilePath = tableRoot + File.separator + relativeFilePath; - Assert.assertEquals( - "If the input is an absolute path, it should be returned as is", - absoluteFilePath, - BaseMigrateDeltaLakeTableAction.getFullFilePath(absoluteFilePath, tableRoot)); - Assert.assertEquals( - "If the input is a relative path, it should be returned as an absolute path", - absoluteFilePath, - BaseMigrateDeltaLakeTableAction.getFullFilePath(relativeFilePath, tableRoot)); - } -} From 5b959250c19bc57f7a8c5122d50e385e6f210ae0 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 10 Jan 2023 03:18:11 -0500 Subject: [PATCH 29/54] refactor the interfaces, add new tests to integration tests, add new unit tests (#40) * rename the interface * add new APIs and add unit test for precondition checks * refactor interface and precondition check * remove redundant private method and refactor javadoc * add test logic for table contains external data files * test the inclusion of delta lake table properties --- .../SnapshotDeltaLakeSparkIntegration.java | 9 +- .../delta/TestSnapshotDeltaLakeTable.java | 123 +++++++++++++++++- .../BaseSnapshotDeltaLakeTableAction.java | 112 +++++++++------- ...akeToIcebergMigrationActionsProvider.java} | 15 +-- .../iceberg/delta/SnapshotDeltaLakeTable.java | 27 ++++ .../TestBaseSnapshotDeltaLakeTableAction.java | 122 +++++++++++++++++ 6 files changed, 342 insertions(+), 66 deletions(-) rename delta-lake/src/main/java/org/apache/iceberg/delta/{SupportSnapshotFromDeltaLake.java => DeltaLakeToIcebergMigrationActionsProvider.java} (61%) create mode 100644 delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java index 90c4867979db..caf4896b2395 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java @@ -33,10 +33,9 @@ static SnapshotDeltaLakeTable snapshotDeltaLakeTable( CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); Spark3Util.CatalogAndIdentifier catalogAndIdent = Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new BaseSnapshotDeltaLakeTableAction( - Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name()), - deltaTableLocation, - TableIdentifier.parse(catalogAndIdent.identifier().toString()), - spark.sessionState().newHadoopConf()); + return new BaseSnapshotDeltaLakeTableAction(deltaTableLocation) + .as(TableIdentifier.parse(catalogAndIdent.identifier().toString())) + .deltaLakeConfiguration(spark.sessionState().newHadoopConf()) + .icebergCatalog(Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name())); } } diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 938e239e60dc..c0134c5a2ce5 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -19,11 +19,18 @@ package org.apache.iceberg.delta; import io.delta.standalone.DeltaLog; +import io.delta.standalone.Operation; +import io.delta.standalone.OptimisticTransaction; +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.actions.RemoveFile; +import io.delta.standalone.exceptions.DeltaConcurrentModificationException; import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -53,6 +60,7 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String NAMESPACE = "default"; private String partitionedIdentifier; private String unpartitionedIdentifier; + private String externalDataFilesIdentifier; private static final String defaultSparkCatalog = "spark_catalog"; private static final String icebergCatalogName = "iceberg_hive"; @@ -79,13 +87,15 @@ public static Object[][] parameters() { @Rule public TemporaryFolder temp1 = new TemporaryFolder(); @Rule public TemporaryFolder temp2 = new TemporaryFolder(); @Rule public TemporaryFolder temp3 = new TemporaryFolder(); + @Rule public TemporaryFolder temp4 = new TemporaryFolder(); private final String partitionedTableName = "partitioned_table"; private final String unpartitionedTableName = "unpartitioned_table"; - + private final String externalDataFilesTableName = "external_data_files_table"; private String partitionedLocation; private String unpartitionedLocation; private String newIcebergTableLocation; + private String externalDataFilesTableLocation; public TestSnapshotDeltaLakeTable( String catalogName, String implementation, Map config) { @@ -99,18 +109,22 @@ public void before() { File partitionedFolder = temp1.newFolder(); File unpartitionedFolder = temp2.newFolder(); File newIcebergTableFolder = temp3.newFolder(); + File externalDataFilesTableFolder = temp4.newFolder(); partitionedLocation = partitionedFolder.toURI().toString(); unpartitionedLocation = unpartitionedFolder.toURI().toString(); newIcebergTableLocation = newIcebergTableFolder.toURI().toString(); + externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString(); } catch (IOException e) { throw new RuntimeException(e); } partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName); unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName); + externalDataFilesIdentifier = destName(defaultSparkCatalog, externalDataFilesTableName); spark.sql(String.format("DROP TABLE IF EXISTS %s", partitionedIdentifier)); spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier)); // Create a partitioned and unpartitioned table, doing a few inserts on each IntStream.range(0, 3) @@ -133,6 +147,12 @@ public void before() { .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) .option("path", unpartitionedLocation) .saveAsTable(unpartitionedIdentifier); + + df.write() + .format("delta") + .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) + .option("path", externalDataFilesTableLocation) + .saveAsTable(externalDataFilesIdentifier); }); // Delete a record from the table @@ -146,7 +166,7 @@ public void before() { @After public void after() { - // Drop the hive table. + // Drop delta lake tables. spark.sql( String.format( "DROP TABLE IF EXISTS %s", destName(defaultSparkCatalog, partitionedTableName))); @@ -195,12 +215,20 @@ public void testSnapshotWithNewLocation() { @Test public void testSnapshotWithAdditionalProperties() { + // add some properties to the original delta table + spark.sql( + "ALTER TABLE " + + unpartitionedIdentifier + + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties"); SnapshotDeltaLakeTable.Result result = SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) .tableProperty("test1", "test1") - .tableProperties(ImmutableMap.of("test2", "test2", "test3", "test3", "test4", "test4")) + .tableProperties( + ImmutableMap.of( + "test2", "test2", "test3", "test3", "test4", + "test4")) // add additional iceberg table properties .execute(); checkSnapshotIntegrity( @@ -208,10 +236,29 @@ public void testSnapshotWithAdditionalProperties() { checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); checkIcebergTableProperties( newTableIdentifier, - ImmutableMap.of("test1", "test1", "test2", "test2", "test3", "test3", "test4", "test4"), + ImmutableMap.of( + "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3", + "test4", "test4"), unpartitionedLocation); } + @Test + public void testSnapshotTableWithExternalDataFiles() { + // Add parquet files to default.external_data_files_table. The newly added parquet files + // are not at the same location as the table. + addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); + + String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files"); + SnapshotDeltaLakeTable.Result result = + SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, externalDataFilesTableLocation) + .execute(); + checkSnapshotIntegrity( + externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation); + checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); + } + private void checkSnapshotIntegrity( String deltaTableLocation, String deltaTableIdentifier, @@ -259,7 +306,7 @@ private void checkIcebergTableProperties( expectedPropertiesBuilder.putAll(expectedAdditionalProperties); ImmutableMap expectedProperties = expectedPropertiesBuilder.build(); Assert.assertTrue( - "The snapshot iceberg table should have the expected properties, both default and user added ones", + "The snapshot iceberg table should have the expected properties, all in original delta lake table, added by the action and user added ones", icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet())); Assert.assertTrue( "The snapshot iceberg table's property should contains the original location", @@ -267,6 +314,27 @@ private void checkIcebergTableProperties( && icebergTable.properties().get(ORIGINAL_LOCATION_PROP).equals(deltaTableLocation)); } + private void checkDataFilePathsIntegrity( + String icebergTableIdentifier, String deltaTableLocation) { + Table icebergTable = getIcebergTable(icebergTableIdentifier); + DeltaLog deltaLog = DeltaLog.forTable(spark.sessionState().newHadoopConf(), deltaTableLocation); + // checkSnapshotIntegrity already checks the number of data files in the snapshot iceberg table + // equals that in the original delta lake table + List deltaTableDataFilePaths = + deltaLog.update().getAllFiles().stream() + .map(f -> getFullFilePath(f.getPath(), deltaLog.getPath().toString())) + .collect(Collectors.toList()); + icebergTable + .currentSnapshot() + .addedDataFiles(icebergTable.io()) + .forEach( + dataFile -> { + Assert.assertTrue( + "The data file path should be the same as the original delta table", + deltaTableDataFilePaths.contains(dataFile.path().toString())); + }); + } + private Table getIcebergTable(String icebergTableIdentifier) { CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); Spark3Util.CatalogAndIdentifier catalogAndIdent = @@ -282,4 +350,49 @@ private String destName(String catalogName, String dest) { } return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest; } + + /** + * Add parquet files manually to a delta lake table to mock the situation that some data files are + * not in the same location as the delta lake table. The case that {@link AddFile#getPath()} or + * {@link RemoveFile#getPath()} returns absolute path + * + *

The knowing issue make it + * necessary to manually rebuild the AddFile to avoid deserialization error when committing the + * transaction. + */ + private void addExternalDatafiles( + String targetDeltaTableLocation, String sourceDeltaTableLocation) { + DeltaLog targetLog = + DeltaLog.forTable(spark.sessionState().newHadoopConf(), targetDeltaTableLocation); + OptimisticTransaction transaction = targetLog.startTransaction(); + DeltaLog sourceLog = + DeltaLog.forTable(spark.sessionState().newHadoopConf(), sourceDeltaTableLocation); + List newFiles = + sourceLog.update().getAllFiles().stream() + .map( + f -> + AddFile.builder( + getFullFilePath(f.getPath(), sourceLog.getPath().toString()), + f.getPartitionValues(), + f.getSize(), + System.currentTimeMillis(), + true) + .build()) + .collect(Collectors.toList()); + try { + transaction.commit(newFiles, new Operation(Operation.Name.UPDATE), "Delta-Lake/2.2.0"); + } catch (DeltaConcurrentModificationException e) { + // handle exception here + throw new RuntimeException(e); + } + } + + private static String getFullFilePath(String path, String tableRoot) { + URI dataFileUri = URI.create(path); + if (dataFileUri.isAbsolute()) { + return path; + } else { + return tableRoot + File.separator + path; + } + } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index f0bb1652a80d..0d4318a4af43 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -55,6 +55,7 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.OrcMetrics; import org.apache.iceberg.parquet.ParquetUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; 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; @@ -79,12 +80,12 @@ public class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable private static final String ORC_SUFFIX = ".orc"; private final ImmutableMap.Builder additionalPropertiesBuilder = ImmutableMap.builder(); - private final DeltaLog deltaLog; - private final Catalog icebergCatalog; + private DeltaLog deltaLog; + private Catalog icebergCatalog; private final String deltaTableLocation; - private final TableIdentifier newTableIdentifier; + private TableIdentifier newTableIdentifier; private String newTableLocation; - private final HadoopFileIO deltaLakeFileIO; + private HadoopFileIO deltaLakeFileIO; /** * Snapshot a delta lake table to be an iceberg table. The action will read the delta lake table's @@ -93,22 +94,11 @@ public class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable * *

The new table will only be created if the snapshot is successful. * - * @param icebergCatalog the iceberg catalog to create the iceberg table * @param deltaTableLocation the delta lake table's path - * @param newTableIdentifier the identifier of the new iceberg table - * @param deltaLakeConfiguration the hadoop configuration to access the delta lake table */ - public BaseSnapshotDeltaLakeTableAction( - Catalog icebergCatalog, - String deltaTableLocation, - TableIdentifier newTableIdentifier, - Configuration deltaLakeConfiguration) { - this.icebergCatalog = icebergCatalog; + public BaseSnapshotDeltaLakeTableAction(String deltaTableLocation) { this.deltaTableLocation = deltaTableLocation; - this.newTableIdentifier = newTableIdentifier; this.newTableLocation = deltaTableLocation; - this.deltaLog = DeltaLog.forTable(deltaLakeConfiguration, deltaTableLocation); - this.deltaLakeFileIO = new HadoopFileIO(deltaLakeConfiguration); } @Override @@ -129,8 +119,33 @@ public SnapshotDeltaLakeTable tableLocation(String location) { return this; } + @Override + public SnapshotDeltaLakeTable as(TableIdentifier identifier) { + this.newTableIdentifier = identifier; + return this; + } + + @Override + public SnapshotDeltaLakeTable icebergCatalog(Catalog catalog) { + this.icebergCatalog = catalog; + return this; + } + + @Override + public SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf) { + this.deltaLog = DeltaLog.forTable(conf, deltaTableLocation); + this.deltaLakeFileIO = new HadoopFileIO(conf); + return this; + } + @Override public Result execute() { + Preconditions.checkArgument( + icebergCatalog != null && newTableIdentifier != null, + "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier."); + Preconditions.checkArgument( + deltaLog != null && deltaLakeFileIO != null, + "Make sure to configure the action with a valid deltaLakeConfiguration"); io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); Schema schema = convertDeltaLakeSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); @@ -142,7 +157,22 @@ public Result execute() { newTableLocation, destTableProperties(updatedSnapshot, deltaTableLocation)); - long totalDataFiles = copyFromDeltaLakeToIceberg(icebergTransaction); + Iterator 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(); + commitDeltaVersionLogToIcebergTransaction(versionLog, icebergTransaction); + } + long totalDataFiles = + Long.parseLong( + icebergTransaction + .table() + .currentSnapshot() + .summary() + .get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); + icebergTransaction.commitTransaction(); LOG.info( "Successfully loaded Iceberg metadata for {} files in {}", @@ -170,32 +200,26 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { return builder.build(); } - private long copyFromDeltaLakeToIceberg(Transaction transaction) { - Iterator 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(); - commitDeltaVersionLogToIcebergTransaction(versionLog, transaction); - } - - return Long.parseLong( - transaction.table().currentSnapshot().summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); - } - + /** + * Iterate through the {@code VersionLog} to determine the update type and commit the update to + * the given {@code Transaction}. + * + *

There are 3 cases: + * + *

1. AppendFiles - 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) + * + * @param versionLog the delta log version to commit to iceberg table transaction + * @param transaction the iceberg table transaction to commit to + */ private void commitDeltaVersionLogToIcebergTransaction( VersionLog versionLog, Transaction transaction) { List actions = versionLog.getActions(); - // We first need to iterate through to see what kind of transaction this was. There are 3 - // cases: - // 1. AppendFiles - 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 Map> deltaLakeActionMap = actions.stream() @@ -254,12 +278,10 @@ public DataFile buildDataFileFromAction(Action action, Table table) { } String fullFilePath = getFullFilePath(path, deltaLog.getPath().toString()); - - if (partitionValues == null) { - // For unpartitioned table, the partitionValues should be an empty map rather than null - throw new IllegalArgumentException( - String.format("File %s does not specify a partitionValues", fullFilePath)); - } + // For unpartitioned table, the partitionValues should be an empty map rather than null + Preconditions.checkArgument( + partitionValues != null, + String.format("File %s does not specify a partitionValues", fullFilePath)); FileFormat format = determineFileFormatFromPath(fullFilePath); InputFile file = deltaLakeFileIO.newInputFile(fullFilePath); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java similarity index 61% rename from delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java rename to delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java index bb2eb4c0fb9c..845e32548d1a 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SupportSnapshotFromDeltaLake.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java @@ -22,17 +22,10 @@ * An API that should be implemented by query engine integrations that want to support snapshotting * from Delta Lake table to Iceberg table. */ -public interface SupportSnapshotFromDeltaLake { +public interface DeltaLakeToIcebergMigrationActionsProvider { - /** Initiates an action to migrate an existing Delta Lake table to Iceberg. */ - default SnapshotDeltaLakeTable migrateDeltaLakeTable(String tableIdent, String deltaS3Location) { - throw new UnsupportedOperationException( - this.getClass().getName() + " does not implement migrateDeltaLakeTable"); - } - - default SnapshotDeltaLakeTable migrateDeltaLakeTable( - String tableIdent, String deltaS3Location, String newTableLocation) { - throw new UnsupportedOperationException( - this.getClass().getName() + " does not implement migrateDeltaLakeTable"); + /** Initiates an action to snapshot an existing Delta Lake table to an Iceberg table. */ + default SnapshotDeltaLakeTable snapshotDeltaLakeTable(String sourceTableLocation) { + return new BaseSnapshotDeltaLakeTableAction(sourceTableLocation); } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java b/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java index 6884da4f1680..6f4bd726a58b 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/SnapshotDeltaLakeTable.java @@ -19,7 +19,10 @@ package org.apache.iceberg.delta; import java.util.Map; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.actions.Action; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; /** Snapshot an existing Delta Lake table to Iceberg in place. */ public interface SnapshotDeltaLakeTable @@ -53,6 +56,30 @@ public interface SnapshotDeltaLakeTable */ SnapshotDeltaLakeTable tableLocation(String location); + /** + * Sets the identifier of the newly created Iceberg table. This is required to be set before + * execute the action. + * + * @param identifier a table identifier (namespace, name) @Returns this for method chaining + */ + SnapshotDeltaLakeTable as(TableIdentifier identifier); + + /** + * Sets the catalog of the newly created Iceberg table. This is required to be set before execute + * the action + * + * @param catalog a catalog @Returns this for method chaining + */ + SnapshotDeltaLakeTable icebergCatalog(Catalog catalog); + + /** + * Sets the Hadoop configuration used to access delta lake table's logs and datafiles. This is + * required to be set before execute the action. + * + * @param conf a Hadoop configuration @Returns this for method chaining + */ + SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf); + /** The action result that contains a summary of the execution. */ interface Result { diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java new file mode 100644 index 000000000000..e210dfeba336 --- /dev/null +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestBaseSnapshotDeltaLakeTableAction { + @Rule public TemporaryFolder temp1 = new TemporaryFolder(); + @Rule public TemporaryFolder temp2 = new TemporaryFolder(); + private String sourceTableLocation; + private final Configuration testHadoopConf = new Configuration(); + private String newTableLocation; + private final Catalog testCatalog = new TestCatalog(); + + @Before + public void before() { + try { + File sourceFolder = temp1.newFolder(); + File destFolder = temp2.newFolder(); + sourceTableLocation = sourceFolder.toURI().toString(); + newTableLocation = destFolder.toURI().toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testRequiredTableIdentifier() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeTableAction(sourceTableLocation) + .icebergCatalog(testCatalog) + .deltaLakeConfiguration(testHadoopConf) + .tableLocation(newTableLocation); + Assert.assertThrows( + "Should throw IllegalArgumentException if table identifier is not set", + IllegalArgumentException.class, + testAction::execute); + } + + @Test + public void testRequiredIcebergCatalog() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .deltaLakeConfiguration(testHadoopConf) + .tableLocation(newTableLocation); + + Assert.assertThrows( + "Should throw IllegalArgumentException if iceberg catalog is not set", + IllegalArgumentException.class, + testAction::execute); + } + + @Test + public void testRequiredDeltaLakeConfiguration() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + Assert.assertThrows( + "Should throw IllegalArgumentException if delta lake configuration is not set", + IllegalArgumentException.class, + testAction::execute); + } + + private static class TestCatalog extends BaseMetastoreCatalog { + TestCatalog() {} + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + return null; + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { + return null; + } + + @Override + public List listTables(Namespace namespace) { + return null; + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + return false; + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) {} + } +} From f43c3256c50709ce2e8784c6c86539944920446b Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 10 Jan 2023 15:39:45 -0500 Subject: [PATCH 30/54] fix error messages and add default implementation for actionProvider (#41) * fix error messages * add getDefault implementation to the action provider --- ...ltaLakeToIcebergMigrationSparkIntegration.java} | 8 +++++--- .../iceberg/delta/TestSnapshotDeltaLakeTable.java | 14 +++++++------- .../delta/BaseSnapshotDeltaLakeTableAction.java | 11 ++++++----- ...DeltaLakeToIcebergMigrationActionsProvider.java | 9 +++++++++ 4 files changed, 27 insertions(+), 15 deletions(-) rename delta-lake/src/integration/java/org/apache/iceberg/delta/{SnapshotDeltaLakeSparkIntegration.java => DeltaLakeToIcebergMigrationSparkIntegration.java} (84%) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java similarity index 84% rename from delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java rename to delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java index caf4896b2395..bac607a772ed 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SnapshotDeltaLakeSparkIntegration.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java @@ -23,9 +23,10 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.CatalogPlugin; -class SnapshotDeltaLakeSparkIntegration { +/** An example class shows how to use the delta lake migration actions in SparkContext. */ +class DeltaLakeToIcebergMigrationSparkIntegration { - private SnapshotDeltaLakeSparkIntegration() {} + private DeltaLakeToIcebergMigrationSparkIntegration() {} static SnapshotDeltaLakeTable snapshotDeltaLakeTable( SparkSession spark, String newTableIdentifier, String deltaTableLocation) { @@ -33,7 +34,8 @@ static SnapshotDeltaLakeTable snapshotDeltaLakeTable( CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); Spark3Util.CatalogAndIdentifier catalogAndIdent = Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return new BaseSnapshotDeltaLakeTableAction(deltaTableLocation) + return DeltaLakeToIcebergMigrationActionsProvider.getDefault() + .snapshotDeltaLakeTable(deltaTableLocation) .as(TableIdentifier.parse(catalogAndIdent.identifier().toString())) .deltaLakeConfiguration(spark.sessionState().newHadoopConf()) .icebergCatalog(Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name())); diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index c0134c5a2ce5..859032c752d2 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -179,7 +179,7 @@ public void after() { public void testBasicSnapshotPartitioned() { String newTableIdentifier = destName(icebergCatalogName, "iceberg_table"); SnapshotDeltaLakeTable.Result result = - SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) .execute(); @@ -191,7 +191,7 @@ public void testBasicSnapshotPartitioned() { public void testBasicSnapshotUnpartitioned() { String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned"); SnapshotDeltaLakeTable.Result result = - SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) .execute(); @@ -204,7 +204,7 @@ public void testBasicSnapshotUnpartitioned() { public void testSnapshotWithNewLocation() { String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location"); SnapshotDeltaLakeTable.Result result = - SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) .tableLocation(newIcebergTableLocation) .execute(); @@ -222,7 +222,7 @@ public void testSnapshotWithAdditionalProperties() { + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties"); SnapshotDeltaLakeTable.Result result = - SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) .tableProperty("test1", "test1") .tableProperties( @@ -250,7 +250,7 @@ public void testSnapshotTableWithExternalDataFiles() { String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files"); SnapshotDeltaLakeTable.Result result = - SnapshotDeltaLakeSparkIntegration.snapshotDeltaLakeTable( + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, externalDataFilesTableLocation) .execute(); checkSnapshotIntegrity( @@ -354,9 +354,9 @@ private String destName(String catalogName, String dest) { /** * Add parquet files manually to a delta lake table to mock the situation that some data files are * not in the same location as the delta lake table. The case that {@link AddFile#getPath()} or - * {@link RemoveFile#getPath()} returns absolute path + * {@link RemoveFile#getPath()} returns absolute path. * - *

The knowing issue make it + *

The known issue make it * necessary to manually rebuild the AddFile to avoid deserialization error when committing the * transaction. */ diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 0d4318a4af43..701edec00677 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -175,9 +175,10 @@ public Result execute() { icebergTransaction.commitTransaction(); LOG.info( - "Successfully loaded Iceberg metadata for {} files in {}", - totalDataFiles, - deltaTableLocation); + "Successfully created Iceberg table {} from delta lake table at {}, total data file count: {}", + newTableIdentifier, + deltaTableLocation, + totalDataFiles); return new BaseSnapshotDeltaLakeTableActionResult(totalDataFiles); } @@ -314,7 +315,7 @@ private FileFormat determineFileFormatFromPath(String path) { } else if (path.endsWith(ORC_SUFFIX)) { return FileFormat.ORC; } else { - throw new ValidationException("The format of the file %s is unsupported", path); + throw new ValidationException("Cannot determine file format from path %s", path); } } @@ -329,7 +330,7 @@ private Metrics getMetricsForFile( case ORC: return OrcMetrics.fromInputFile(file, metricsSpec, mapping); default: - throw new ValidationException("Unsupported file format: %s", format); + throw new ValidationException("Cannot get metrics from file format: %s", format); } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java index 845e32548d1a..906433ee0268 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java @@ -28,4 +28,13 @@ public interface DeltaLakeToIcebergMigrationActionsProvider { default SnapshotDeltaLakeTable snapshotDeltaLakeTable(String sourceTableLocation) { return new BaseSnapshotDeltaLakeTableAction(sourceTableLocation); } + + static DeltaLakeToIcebergMigrationActionsProvider getDefault() { + return new DefaultDeltaLakeToIcebergMigrationActions(); + } + + class DefaultDeltaLakeToIcebergMigrationActions + implements DeltaLakeToIcebergMigrationActionsProvider { + DefaultDeltaLakeToIcebergMigrationActions() {} + } } From b2a8bfe9f2a5b3497f780cc22da7c8ff3c8e796b Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 10 Jan 2023 18:36:38 -0500 Subject: [PATCH 31/54] refactor the default implementation and javadoc (#43) * rename default implementation and make it an instance * optimize javadoc * make base classes package-private * refactor javadoc in the interface --- ...akeToIcebergMigrationSparkIntegration.java | 11 ++++++- .../BaseSnapshotDeltaLakeTableAction.java | 4 +-- ...aseSnapshotDeltaLakeTableActionResult.java | 4 +-- ...LakeToIcebergMigrationActionsProvider.java | 30 +++++++++++++++---- 4 files changed, 38 insertions(+), 11 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java index bac607a772ed..734ebf5e965b 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java @@ -28,13 +28,22 @@ class DeltaLakeToIcebergMigrationSparkIntegration { private DeltaLakeToIcebergMigrationSparkIntegration() {} + /** + * Example of creating a snapshot a delta table to iceberg table action in SparkContext. + * + * @param spark a SparkSession with iceberg catalog configured. + * @param newTableIdentifier can be both 2 parts and 3 parts identifier, if it is 2 parts, the + * default spark catalog will be used + * @param deltaTableLocation the location of the delta table + * @return an instance of snapshot delta lake table action. + */ static SnapshotDeltaLakeTable snapshotDeltaLakeTable( SparkSession spark, String newTableIdentifier, String deltaTableLocation) { String ctx = "delta lake snapshot target"; CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); Spark3Util.CatalogAndIdentifier catalogAndIdent = Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); - return DeltaLakeToIcebergMigrationActionsProvider.getDefault() + return DeltaLakeToIcebergMigrationActionsProvider.defaultActions() .snapshotDeltaLakeTable(deltaTableLocation) .as(TableIdentifier.parse(catalogAndIdent.identifier().toString())) .deltaLakeConfiguration(spark.sessionState().newHadoopConf()) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 701edec00677..27ef6624cfab 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -68,7 +68,7 @@ * optional user-specified location (default to the Delta Lake table's location) with a different * identifier. */ -public class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { +class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { private static final Logger LOG = LoggerFactory.getLogger(BaseSnapshotDeltaLakeTableAction.class); @@ -96,7 +96,7 @@ public class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable * * @param deltaTableLocation the delta lake table's path */ - public BaseSnapshotDeltaLakeTableAction(String deltaTableLocation) { + BaseSnapshotDeltaLakeTableAction(String deltaTableLocation) { this.deltaTableLocation = deltaTableLocation; this.newTableLocation = deltaTableLocation; } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java index c618e5d6dc47..53c9b0d7fe8b 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableActionResult.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.delta; -public class BaseSnapshotDeltaLakeTableActionResult implements SnapshotDeltaLakeTable.Result { +class BaseSnapshotDeltaLakeTableActionResult implements SnapshotDeltaLakeTable.Result { private final long snapshotDataFilesCount; - public BaseSnapshotDeltaLakeTableActionResult(long snapshotDataFilesCount) { + BaseSnapshotDeltaLakeTableActionResult(long snapshotDataFilesCount) { this.snapshotDataFilesCount = snapshotDataFilesCount; } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java index 906433ee0268..8699eb3b5d26 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java @@ -19,22 +19,40 @@ package org.apache.iceberg.delta; /** - * An API that should be implemented by query engine integrations that want to support snapshotting - * from Delta Lake table to Iceberg table. + * An API that provide actions for migration from a delta lake table to an iceberg table. Query + * engines can use {@code defaultActions()} to access default action implementations, or implement + * this provider to supply a different implementation if necessary. */ public interface DeltaLakeToIcebergMigrationActionsProvider { - /** Initiates an action to snapshot an existing Delta Lake table to an Iceberg table. */ + /** + * Initiates an action to snapshot an existing Delta Lake table to an Iceberg table. + * + * @param sourceTableLocation the location of the Delta Lake table + * @return a {@link SnapshotDeltaLakeTable} action + */ default SnapshotDeltaLakeTable snapshotDeltaLakeTable(String sourceTableLocation) { return new BaseSnapshotDeltaLakeTableAction(sourceTableLocation); } - static DeltaLakeToIcebergMigrationActionsProvider getDefault() { - return new DefaultDeltaLakeToIcebergMigrationActions(); + /** + * Get the default implementation of {@link DeltaLakeToIcebergMigrationActionsProvider} + * + * @return an instance with access to all default actions + */ + static DeltaLakeToIcebergMigrationActionsProvider defaultActions() { + return DefaultDeltaLakeToIcebergMigrationActions.defaultMigrationActions(); } class DefaultDeltaLakeToIcebergMigrationActions implements DeltaLakeToIcebergMigrationActionsProvider { - DefaultDeltaLakeToIcebergMigrationActions() {} + private static final DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions = + new DefaultDeltaLakeToIcebergMigrationActions(); + + private DefaultDeltaLakeToIcebergMigrationActions() {} + + static DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions() { + return defaultMigrationActions; + } } } From 450a08c0c75f8b478fae1b4e355b563cb5329191 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 02:06:41 -0500 Subject: [PATCH 32/54] fix error when migrating table with nested fields, add CI, upgrade test (#44) * fix parquet import error for nested schema * add delta conversion CI * upgrade the test --- .github/workflows/delta-conversion-ci.yml | 115 +++++++++++++++ .../delta/TestSnapshotDeltaLakeTable.java | 135 +++++++++++++----- .../BaseSnapshotDeltaLakeTableAction.java | 9 +- 3 files changed, 226 insertions(+), 33 deletions(-) create mode 100644 .github/workflows/delta-conversion-ci.yml diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml new file mode 100644 index 000000000000..05632108513b --- /dev/null +++ b/.github/workflows/delta-conversion-ci.yml @@ -0,0 +1,115 @@ +# +# 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. +# + +name: "Delta Conversion CI" +on: + push: + branches: + - 'master' + - '0.**' + tags: + - 'apache-iceberg-**' + pull_request: + paths-ignore: + - '.github/ISSUE_TEMPLATE/iceberg_bug_report.yml' + - '.github/workflows/python-ci.yml' + - '.github/workflows/flink-ci.yml' + - '.github/workflows/hive-ci.yml' + - '.gitignore' + - '.asf.yml' + - 'dev/**' + - 'mr/**' + - 'hive3/**' + - 'hive3-orc-bundle/**' + - 'hive-runtime/**' + - 'flink/**' + - 'pig/**' + - 'python/**' + - 'python_legacy/**' + - 'docs/**' + - 'open-api/**' + - 'format/**' + - '.gitattributes' + - 'README.md' + - 'CONTRIBUTING.md' + - 'LICENSE' + - 'NOTICE' + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: ${{ github.event_name == 'pull_request' }} + +jobs: + delta-conversion-scala-2-12-tests: + runs-on: ubuntu-20.04 + strategy: + matrix: + jvm: [8, 11] + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v3 + - uses: actions/setup-java@v3 + with: + distribution: zulu + java-version: ${{ matrix.jvm }} + - uses: actions/cache@v3 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:integrationTest -Pquick=true -x javadoc + - uses: actions/upload-artifact@v3 + if: failure() + with: + name: test logs + path: | + **/build/testlogs + + delta-conversion-scala-2-13-tests: + runs-on: ubuntu-20.04 + strategy: + matrix: + jvm: [ 8, 11 ] + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v3 + - uses: actions/setup-java@v3 + with: + distribution: zulu + java-version: ${{ matrix.jvm }} + - uses: actions/cache@v3 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:integrationTest -Pquick=true -x javadoc + - uses: actions/upload-artifact@v3 + if: failure() + with: + name: test logs + path: | + **/build/testlogs diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 859032c752d2..1ad494029307 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -29,17 +29,18 @@ import java.net.URI; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.IntStream; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.delta.catalog.DeltaCatalog; @@ -54,6 +55,29 @@ @RunWith(Parameterized.class) public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { + private static final String row1 = + "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," + + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}}," + + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"]," + + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}"; + private static final String row2 = + "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}]," + + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"}," + + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}"; + private static final String row3 = + "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," + + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":" + + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}"; + private static final String row4 = + "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," + + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}}," + + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":" + + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}"; + private static final String row5 = + "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," + + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}}," + + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"]," + + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}"; private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; @@ -103,6 +127,50 @@ public TestSnapshotDeltaLakeTable( spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); } + /** + * The test hardcode a nested dataframe to test the snapshot feature. The schema of created + * dataframe is: + * + *

+   *  root
+   *  |-- address_nested: struct (nullable = true)
+   *  |    |-- current: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |    |-- previous: struct (nullable = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- addresses: array (nullable = true)
+   *  |    |-- element: struct (containsNull = true)
+   *  |    |    |-- city: string (nullable = true)
+   *  |    |    |-- state: string (nullable = true)
+   *  |-- id: long (nullable = true)
+   *  |-- magic_number: double (nullable = true)
+   *  |-- name: string (nullable = true)
+   *  |-- properties: struct (nullable = true)
+   *  |    |-- eye: string (nullable = true)
+   *  |    |-- hair: string (nullable = true)
+   *  |-- secondProp: struct (nullable = true)
+   *  |    |-- height: string (nullable = true)
+   *  |-- subjects: array (nullable = true)
+   *  |    |-- element: array (containsNull = true)
+   *  |    |    |-- element: string (containsNull = true)
+   * 
+ * + * The dataframe content is (by calling df.show()): + * + *
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
+   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
+   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
+   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
+   * 
+ */ @Before public void before() { try { @@ -126,38 +194,41 @@ public void before() { spark.sql(String.format("DROP TABLE IF EXISTS %s", unpartitionedIdentifier)); spark.sql(String.format("DROP TABLE IF EXISTS %s", externalDataFilesIdentifier)); - // Create a partitioned and unpartitioned table, doing a few inserts on each - IntStream.range(0, 3) - .forEach( - i -> { - List record = - Lists.newArrayList(new SimpleRecord(i, UUID.randomUUID().toString())); - - Dataset df = spark.createDataFrame(record, SimpleRecord.class); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .partitionBy("id") - .option("path", partitionedLocation) - .saveAsTable(partitionedIdentifier); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .option("path", unpartitionedLocation) - .saveAsTable(unpartitionedIdentifier); - - df.write() - .format("delta") - .mode(i == 0 ? SaveMode.Overwrite : SaveMode.Append) - .option("path", externalDataFilesTableLocation) - .saveAsTable(externalDataFilesIdentifier); - }); + // hard code the dataframe + List jsonList = Lists.newArrayList(); + jsonList.add(row1); + jsonList.add(row2); + jsonList.add(row3); + jsonList.add(row4); + jsonList.add(row5); + JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + SQLContext sqlContext = new SQLContext(javaSparkContext); + JavaRDD rdd = javaSparkContext.parallelize(jsonList); + Dataset df = sqlContext.read().json(rdd); + + // write to delta tables + df.write() + .format("delta") + .mode(SaveMode.Append) + .partitionBy("id") + .option("path", partitionedLocation) + .saveAsTable(partitionedIdentifier); + + df.write() + .format("delta") + .mode(SaveMode.Append) + .option("path", unpartitionedLocation) + .saveAsTable(unpartitionedIdentifier); + + df.write() + .format("delta") + .mode(SaveMode.Append) + .option("path", externalDataFilesTableLocation) + .saveAsTable(externalDataFilesIdentifier); // Delete a record from the table - spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=0"); - spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=0"); + 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"); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 27ef6624cfab..71a05cc9909d 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -51,6 +51,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.OrcMetrics; @@ -156,7 +157,13 @@ public Result execute() { partitionSpec, newTableLocation, destTableProperties(updatedSnapshot, deltaTableLocation)); - + icebergTransaction + .table() + .updateProperties() + .set( + TableProperties.DEFAULT_NAME_MAPPING, + NameMappingParser.toJson(MappingUtil.create(icebergTransaction.table().schema()))) + .commit(); Iterator versionLogIterator = deltaLog.getChanges( 0, // retrieve actions starting from the initial version From 300d39b4705b8694a276e30b5e6906142b0d5b6e Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 02:27:52 -0500 Subject: [PATCH 33/54] remove unused helper class in test --- .../apache/iceberg/delta/SimpleRecord.java | 78 ------------------- 1 file changed, 78 deletions(-) delete mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java deleted file mode 100644 index 7a35bd581060..000000000000 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SimpleRecord.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.delta; - -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -public class SimpleRecord { - private Integer id; - private String data; - - public SimpleRecord() {} - - public SimpleRecord(Integer id, String data) { - this.id = id; - this.data = data; - } - - public Integer getId() { - return id; - } - - public void setId(Integer id) { - this.id = id; - } - - public String getData() { - return data; - } - - public void setData(String data) { - this.data = data; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SimpleRecord record = (SimpleRecord) o; - return Objects.equal(id, record.id) && Objects.equal(data, record.data); - } - - @Override - public int hashCode() { - return Objects.hashCode(id, data); - } - - @Override - public String toString() { - StringBuilder buffer = new StringBuilder(); - buffer.append("{\"id\"="); - buffer.append(id); - buffer.append(",\"data\"=\""); - buffer.append(data); - buffer.append("\"}"); - return buffer.toString(); - } -} From a285c4adbc865ab66ff23e053c05369374daf1e2 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 18:15:37 -0500 Subject: [PATCH 34/54] add null check for stopMetastoreAndSpark, remove unnecessary try-catch --- .../delta/SparkDeltaLakeSnapshotTestBase.java | 12 ++++++---- .../delta/TestSnapshotDeltaLakeTable.java | 22 ++++++++----------- .../iceberg/delta/DeltaLakeTypeToType.java | 1 - 3 files changed, 17 insertions(+), 18 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java index 59bc7626dff4..1e291f3759c3 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java @@ -90,10 +90,14 @@ public static void startMetastoreAndSpark() { @AfterClass public static void stopMetastoreAndSpark() throws Exception { SparkDeltaLakeSnapshotTestBase.catalog = null; - metastore.stop(); - SparkDeltaLakeSnapshotTestBase.metastore = null; - spark.stop(); - SparkDeltaLakeSnapshotTestBase.spark = null; + if (metastore != null) { + metastore.stop(); + SparkDeltaLakeSnapshotTestBase.metastore = null; + } + if (spark != null) { + spark.stop(); + SparkDeltaLakeSnapshotTestBase.spark = null; + } } @BeforeClass diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 1ad494029307..f6f696f9bc82 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -172,19 +172,15 @@ public TestSnapshotDeltaLakeTable( * */ @Before - public void before() { - try { - File partitionedFolder = temp1.newFolder(); - File unpartitionedFolder = temp2.newFolder(); - File newIcebergTableFolder = temp3.newFolder(); - File externalDataFilesTableFolder = temp4.newFolder(); - partitionedLocation = partitionedFolder.toURI().toString(); - unpartitionedLocation = unpartitionedFolder.toURI().toString(); - newIcebergTableLocation = newIcebergTableFolder.toURI().toString(); - externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString(); - } catch (IOException e) { - throw new RuntimeException(e); - } + public void before() throws IOException { + File partitionedFolder = temp1.newFolder(); + File unpartitionedFolder = temp2.newFolder(); + File newIcebergTableFolder = temp3.newFolder(); + File externalDataFilesTableFolder = temp4.newFolder(); + partitionedLocation = partitionedFolder.toURI().toString(); + unpartitionedLocation = unpartitionedFolder.toURI().toString(); + newIcebergTableLocation = newIcebergTableFolder.toURI().toString(); + externalDataFilesTableLocation = externalDataFilesTableFolder.toURI().toString(); partitionedIdentifier = destName(defaultSparkCatalog, partitionedTableName); unpartitionedIdentifier = destName(defaultSparkCatalog, unpartitionedTableName); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java index e0bb5a7a7342..752fb11ba04c 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeTypeToType.java @@ -78,7 +78,6 @@ public Type struct(StructType struct, List types) { id = getNextId(); } - // TODO: refactor const string String doc = field.getMetadata().contains("comment") ? field.getMetadata().get("comment").toString() From 5760a83a712193b1fb0e0c650e4b67ee8c775c79 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 18:29:00 -0500 Subject: [PATCH 35/54] use assertions.assertThatThrownBy to test precondition checks --- .../TestBaseSnapshotDeltaLakeTableAction.java | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java index e210dfeba336..383f1680dda8 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -27,7 +27,7 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.Assert; +import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -60,10 +60,10 @@ public void testRequiredTableIdentifier() { .icebergCatalog(testCatalog) .deltaLakeConfiguration(testHadoopConf) .tableLocation(newTableLocation); - Assert.assertThrows( - "Should throw IllegalArgumentException if table identifier is not set", - IllegalArgumentException.class, - testAction::execute); + Assertions.assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier."); } @Test @@ -73,11 +73,10 @@ public void testRequiredIcebergCatalog() { .as(TableIdentifier.of("test", "test")) .deltaLakeConfiguration(testHadoopConf) .tableLocation(newTableLocation); - - Assert.assertThrows( - "Should throw IllegalArgumentException if iceberg catalog is not set", - IllegalArgumentException.class, - testAction::execute); + Assertions.assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier."); } @Test @@ -87,10 +86,9 @@ public void testRequiredDeltaLakeConfiguration() { .as(TableIdentifier.of("test", "test")) .icebergCatalog(testCatalog) .tableLocation(newTableLocation); - Assert.assertThrows( - "Should throw IllegalArgumentException if delta lake configuration is not set", - IllegalArgumentException.class, - testAction::execute); + Assertions.assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Make sure to configure the action with a valid deltaLakeConfiguration"); } private static class TestCatalog extends BaseMetastoreCatalog { From e41c787cbd5844255004438708555a2e7e91b9f4 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 18:53:50 -0500 Subject: [PATCH 36/54] use assertThat to fix assert.True in TestDeltaLakeTypeToType --- .../delta/TestDeltaLakeTypeToType.java | 87 +++++++++---------- 1 file changed, 42 insertions(+), 45 deletions(-) diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 44f270cd911f..fb7166fca223 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -29,7 +29,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.junit.Assert; +import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; @@ -67,18 +67,13 @@ public void testAtomicTypeConversion() { DeltaLakeDataTypeVisitor.visit( deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - Assert.assertTrue( - "The BooleanType should be converted to BooleanType", - convertedSchema.findType(optionalBooleanType) instanceof Types.BooleanType); - Assert.assertTrue( - "The converted BooleanType field is optional", - convertedSchema.findField(optionalBooleanType).isOptional()); - Assert.assertTrue( - "The BinaryType is converted to BinaryType", - convertedSchema.findType(requiredBinaryType) instanceof Types.BinaryType); - Assert.assertTrue( - "The converted BinaryType field is required", - convertedSchema.findField(requiredBinaryType).isRequired()); + + Assertions.assertThat(convertedSchema.findType(optionalBooleanType)) + .isInstanceOf(Types.BooleanType.class); + Assertions.assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue(); + Assertions.assertThat(convertedSchema.findType(requiredBinaryType)) + .isInstanceOf(Types.BinaryType.class); + Assertions.assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue(); } @Test @@ -87,38 +82,40 @@ public void testNestedTypeConversion() { DeltaLakeDataTypeVisitor.visit( deltaNestedSchema, new DeltaLakeTypeToType(deltaNestedSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - Assert.assertTrue( - "The StructType is converted to StructType", - convertedSchema.findType(innerAtomicSchema) instanceof Types.StructType); - Assert.assertTrue( - "The converted StructType contains subfield BooleanType", - convertedSchema.findType(innerAtomicSchema).asStructType().fieldType(optionalBooleanType) - instanceof Types.BooleanType); - Assert.assertTrue( - "The converted StructType contains subfield BinaryType", - convertedSchema.findType(innerAtomicSchema).asStructType().fieldType(requiredBinaryType) - instanceof Types.BinaryType); - - Assert.assertTrue( - "The MapType is converted to MapType", - convertedSchema.findType(stringLongMapType) instanceof Types.MapType); - Assert.assertTrue( - "The converted MapType has key as StringType", - convertedSchema.findType(stringLongMapType).asMapType().keyType() - instanceof Types.StringType); - Assert.assertTrue( - "The converted MapType has value as LongType", - convertedSchema.findType(stringLongMapType).asMapType().valueType() - instanceof Types.LongType); - Assert.assertTrue( - "The ArrayType is converted to ListType", - convertedSchema.findType(doubleArrayType) instanceof Types.ListType); - Assert.assertTrue( - "The converted ListType field is required", - convertedSchema.findField(doubleArrayType).isRequired()); - Assert.assertTrue( - "The converted ListType field contains optional doubleType element", - convertedSchema.findType(doubleArrayType).asListType().isElementOptional()); + Assertions.assertThat(convertedSchema.findType(innerAtomicSchema)) + .isInstanceOf(Types.StructType.class); + Assertions.assertThat(convertedSchema.findField(innerAtomicSchema).isOptional()).isTrue(); + Assertions.assertThat( + convertedSchema + .findType(innerAtomicSchema) + .asStructType() + .fieldType(optionalBooleanType)) + .isInstanceOf(Types.BooleanType.class); + Assertions.assertThat( + convertedSchema + .findType(innerAtomicSchema) + .asStructType() + .fieldType(requiredBinaryType)) + .isInstanceOf(Types.BinaryType.class); + Assertions.assertThat( + convertedSchema + .findType(innerAtomicSchema) + .asStructType() + .field(requiredBinaryType) + .isRequired()) + .isTrue(); + Assertions.assertThat(convertedSchema.findType(stringLongMapType)) + .isInstanceOf(Types.MapType.class); + Assertions.assertThat(convertedSchema.findType(stringLongMapType).asMapType().keyType()) + .isInstanceOf(Types.StringType.class); + Assertions.assertThat(convertedSchema.findType(stringLongMapType).asMapType().valueType()) + .isInstanceOf(Types.LongType.class); + Assertions.assertThat(convertedSchema.findType(doubleArrayType)) + .isInstanceOf(Types.ListType.class); + Assertions.assertThat(convertedSchema.findField(doubleArrayType).isRequired()).isTrue(); + Assertions.assertThat( + convertedSchema.findType(doubleArrayType).asListType().isElementOptional()) + .isTrue(); } } From 7a168098e91a4f8a600e5f7ea4b80411bcd8eb63 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 19:23:38 -0500 Subject: [PATCH 37/54] use AssertionJ in TestSnapshotDeltaLakeTable --- .../delta/TestSnapshotDeltaLakeTable.java | 25 +++++++------------ 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index f6f696f9bc82..b5f07015e5f9 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -44,6 +44,7 @@ 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.Assert; import org.junit.Before; @@ -342,16 +343,12 @@ private void checkSnapshotIntegrity( "The original table and the transformed one should have the same size", deltaTableContents.size(), icebergTableContents.size()); - Assert.assertTrue( - "The original table and the transformed one should have the same contents", - icebergTableContents.containsAll(deltaTableContents)); - Assert.assertTrue( - "The original table and the transformed one should have the same contents", - deltaTableContents.containsAll(icebergTableContents)); Assert.assertEquals( "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table", deltaLog.update().getAllFiles().size(), snapshotReport.snapshotDataFilesCount()); + Assertions.assertThat(icebergTableContents).containsAll(deltaTableContents); + Assertions.assertThat(deltaTableContents).containsAll(icebergTableContents); } private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) { @@ -372,13 +369,11 @@ private void checkIcebergTableProperties( expectedPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE); expectedPropertiesBuilder.putAll(expectedAdditionalProperties); ImmutableMap expectedProperties = expectedPropertiesBuilder.build(); - Assert.assertTrue( - "The snapshot iceberg table should have the expected properties, all in original delta lake table, added by the action and user added ones", - icebergTable.properties().entrySet().containsAll(expectedProperties.entrySet())); - Assert.assertTrue( - "The snapshot iceberg table's property should contains the original location", - icebergTable.properties().containsKey(ORIGINAL_LOCATION_PROP) - && icebergTable.properties().get(ORIGINAL_LOCATION_PROP).equals(deltaTableLocation)); + + Assertions.assertThat(icebergTable.properties().entrySet()) + .containsAll(expectedProperties.entrySet()); + Assertions.assertThat(icebergTable.properties()) + .containsEntry(ORIGINAL_LOCATION_PROP, deltaTableLocation); } private void checkDataFilePathsIntegrity( @@ -396,9 +391,7 @@ private void checkDataFilePathsIntegrity( .addedDataFiles(icebergTable.io()) .forEach( dataFile -> { - Assert.assertTrue( - "The data file path should be the same as the original delta table", - deltaTableDataFilePaths.contains(dataFile.path().toString())); + Assertions.assertThat(deltaTableDataFilePaths).contains(dataFile.path().toString()); }); } From 7072612f9dbc587cda5eb2bddb1b61f48f85fe84 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 21:14:11 -0500 Subject: [PATCH 38/54] fix format and nit issue --- ...akeToIcebergMigrationSparkIntegration.java | 3 ++- .../delta/TestSnapshotDeltaLakeTable.java | 22 +++++++++---------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java index 734ebf5e965b..538dc708157c 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java @@ -29,7 +29,8 @@ class DeltaLakeToIcebergMigrationSparkIntegration { private DeltaLakeToIcebergMigrationSparkIntegration() {} /** - * Example of creating a snapshot a delta table to iceberg table action in SparkContext. + * Example of how to use a {@link SparkSession}, a table identifier and a delta table location to + * construct an action for snapshotting the delta table to an iceberg table. * * @param spark a SparkSession with iceberg catalog configured. * @param newTableIdentifier can be both 2 parts and 3 parts identifier, if it is 2 parts, the diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index b5f07015e5f9..3f3d9e113c78 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -83,11 +83,18 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; private static final String NAMESPACE = "default"; + private static final String defaultSparkCatalog = "spark_catalog"; + private static final String icebergCatalogName = "iceberg_hive"; private String partitionedIdentifier; private String unpartitionedIdentifier; private String externalDataFilesIdentifier; - private static final String defaultSparkCatalog = "spark_catalog"; - private static final String icebergCatalogName = "iceberg_hive"; + private final String partitionedTableName = "partitioned_table"; + private final String unpartitionedTableName = "unpartitioned_table"; + private final String externalDataFilesTableName = "external_data_files_table"; + private String partitionedLocation; + private String unpartitionedLocation; + private String newIcebergTableLocation; + private String externalDataFilesTableLocation; @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") public static Object[][] parameters() { @@ -114,14 +121,6 @@ public static Object[][] parameters() { @Rule public TemporaryFolder temp3 = new TemporaryFolder(); @Rule public TemporaryFolder temp4 = new TemporaryFolder(); - private final String partitionedTableName = "partitioned_table"; - private final String unpartitionedTableName = "unpartitioned_table"; - private final String externalDataFilesTableName = "external_data_files_table"; - private String partitionedLocation; - private String unpartitionedLocation; - private String newIcebergTableLocation; - private String externalDataFilesTableLocation; - public TestSnapshotDeltaLakeTable( String catalogName, String implementation, Map config) { super(catalogName, implementation, config); @@ -416,7 +415,7 @@ private String destName(String catalogName, String dest) { * not in the same location as the delta lake table. The case that {@link AddFile#getPath()} or * {@link RemoveFile#getPath()} returns absolute path. * - *

The known issue make it + *

The known issue makes it * necessary to manually rebuild the AddFile to avoid deserialization error when committing the * transaction. */ @@ -442,7 +441,6 @@ private void addExternalDatafiles( try { transaction.commit(newFiles, new Operation(Operation.Name.UPDATE), "Delta-Lake/2.2.0"); } catch (DeltaConcurrentModificationException e) { - // handle exception here throw new RuntimeException(e); } } From c2293c9b6bf6514d3d7e4834736c2290ebc0694a Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 22:06:49 -0500 Subject: [PATCH 39/54] remove unnecessary fields and class and let integrationTest collected by CI (#45) * remove unnecessary fields and class * make integration test collected by check --- .github/workflows/delta-conversion-ci.yml | 4 +- build.gradle | 1 + .../iceberg/delta/SparkCatalogConfig.java | 69 ------------------- .../delta/SparkDeltaLakeSnapshotTestBase.java | 36 +--------- 4 files changed, 4 insertions(+), 106 deletions(-) delete mode 100644 delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 05632108513b..5302f048f773 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -77,7 +77,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:integrationTest -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: @@ -106,7 +106,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:integrationTest -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: diff --git a/build.gradle b/build.gradle index d6cf4d88745d..d4121190220c 100644 --- a/build.gradle +++ b/build.gradle @@ -509,6 +509,7 @@ project(':iceberg-delta-lake') { testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath } + check.dependsOn integrationTest } } diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java deleted file mode 100644 index 40631eae2c2b..000000000000 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkCatalogConfig.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.delta; - -import java.util.Map; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.spark.SparkCatalog; -import org.apache.iceberg.spark.SparkSessionCatalog; - -public enum SparkCatalogConfig { - HIVE( - "testhive", - SparkCatalog.class.getName(), - ImmutableMap.of( - "type", "hive", - "default-namespace", "default")), - HADOOP("testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop")), - SPARK( - "spark_catalog", - SparkSessionCatalog.class.getName(), - ImmutableMap.of( - "type", - "hive", - "default-namespace", - "default", - "parquet-enabled", - "true", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - )); - - private final String catalogName; - private final String implementation; - private final Map properties; - - SparkCatalogConfig(String catalogName, String implementation, Map properties) { - this.catalogName = catalogName; - this.implementation = implementation; - this.properties = properties; - } - - public String catalogName() { - return catalogName; - } - - public String implementation() { - return implementation; - } - - public Map properties() { - return properties; - } -} diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java index 1e291f3759c3..daab48516317 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java @@ -28,12 +28,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -44,8 +40,6 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; @SuppressWarnings("VisibilityModifier") public abstract class SparkDeltaLakeSnapshotTestBase { @@ -53,8 +47,8 @@ public abstract class SparkDeltaLakeSnapshotTestBase { protected static HiveConf hiveConf = null; protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - private static File warehouse = null; + protected final String catalogName; @BeforeClass public static void startMetastoreAndSpark() { @@ -115,30 +109,9 @@ public static void dropWarehouse() throws IOException { } } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - protected final String catalogName; - protected final Catalog validationCatalog; - protected final SupportsNamespaces validationNamespaceCatalog; - protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); - protected final String tableName; - - public SparkDeltaLakeSnapshotTestBase() { - this(SparkCatalogConfig.HADOOP); - } - - public SparkDeltaLakeSnapshotTestBase(SparkCatalogConfig config) { - this(config.catalogName(), config.implementation(), config.properties()); - } - public SparkDeltaLakeSnapshotTestBase( String catalogName, String implementation, Map config) { this.catalogName = catalogName; - this.validationCatalog = - catalogName.equals("testhadoop") - ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; spark.conf().set("spark.sql.catalog." + catalogName, implementation); config.forEach( @@ -148,16 +121,9 @@ public SparkDeltaLakeSnapshotTestBase( spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); } - this.tableName = - (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table"; - sql("CREATE NAMESPACE IF NOT EXISTS default"); } - protected String tableName(String name) { - return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default." + name; - } - protected List sql(String query, Object... args) { List rows = spark.sql(String.format(query, args)).collectAsList(); if (rows.size() < 1) { From f38d7b151178e7271aff70eb45312ed6171ec14e Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 12 Jan 2023 22:25:26 -0500 Subject: [PATCH 40/54] remove unnecessary try catch --- .../TestBaseSnapshotDeltaLakeTableAction.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java index 383f1680dda8..941bc4a990b3 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -42,15 +42,11 @@ public class TestBaseSnapshotDeltaLakeTableAction { private final Catalog testCatalog = new TestCatalog(); @Before - public void before() { - try { - File sourceFolder = temp1.newFolder(); - File destFolder = temp2.newFolder(); - sourceTableLocation = sourceFolder.toURI().toString(); - newTableLocation = destFolder.toURI().toString(); - } catch (IOException e) { - throw new RuntimeException(e); - } + public void before() throws IOException { + File sourceFolder = temp1.newFolder(); + File destFolder = temp2.newFolder(); + sourceTableLocation = sourceFolder.toURI().toString(); + newTableLocation = destFolder.toURI().toString(); } @Test From 10163f806710560c0d51827ce5324e17b63dd79c Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Fri, 13 Jan 2023 12:31:30 -0500 Subject: [PATCH 41/54] fix wrong modifier of a private method --- .../apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 71a05cc9909d..8b96461d989a 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -267,7 +267,7 @@ private void commitDeltaVersionLogToIcebergTransaction( } } - public DataFile buildDataFileFromAction(Action action, Table table) { + private DataFile buildDataFileFromAction(Action action, Table table) { PartitionSpec spec = table.spec(); String path; Map partitionValues; From a7c3de152ec6547c0df09de95072670ed03cd6e0 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Mon, 16 Jan 2023 21:54:54 -0500 Subject: [PATCH 42/54] simplify the test base (#46) * remove unnecessary namespace creation * move namespace creation to TestSnapshotDeltaLakeTable.java --- .../delta/SparkDeltaLakeSnapshotTestBase.java | 88 ------------------- .../delta/TestSnapshotDeltaLakeTable.java | 14 ++- 2 files changed, 13 insertions(+), 89 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java index daab48516317..2c100e2fbe95 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java @@ -18,27 +18,12 @@ */ package org.apache.iceberg.delta; -import java.io.File; -import java.io.IOException; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.internal.SQLConf; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; @SuppressWarnings("VisibilityModifier") @@ -46,9 +31,6 @@ public abstract class SparkDeltaLakeSnapshotTestBase { protected static TestHiveMetastore metastore = null; protected static HiveConf hiveConf = null; protected static SparkSession spark = null; - protected static HiveCatalog catalog = null; - private static File warehouse = null; - protected final String catalogName; @BeforeClass public static void startMetastoreAndSpark() { @@ -64,26 +46,13 @@ public static void startMetastoreAndSpark() { "spark.hadoop." + HiveConf.ConfVars.METASTOREURIS.varname, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") - // Needed for Delta Lake tests .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .enableHiveSupport() .getOrCreate(); - - SparkDeltaLakeSnapshotTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - - try { - catalog.createNamespace(Namespace.of("default")); - } catch (AlreadyExistsException ignored) { - // the default namespace already exists. ignore the create error - } } @AfterClass public static void stopMetastoreAndSpark() throws Exception { - SparkDeltaLakeSnapshotTestBase.catalog = null; if (metastore != null) { metastore.stop(); SparkDeltaLakeSnapshotTestBase.metastore = null; @@ -94,68 +63,11 @@ public static void stopMetastoreAndSpark() throws Exception { } } - @BeforeClass - public static void createWarehouse() throws IOException { - SparkDeltaLakeSnapshotTestBase.warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); - } - - @AfterClass - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); - } - } - public SparkDeltaLakeSnapshotTestBase( String catalogName, String implementation, Map config) { - this.catalogName = catalogName; spark.conf().set("spark.sql.catalog." + catalogName, implementation); config.forEach( (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); - - if (config.get("type").equalsIgnoreCase("hadoop")) { - spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); - } - - sql("CREATE NAMESPACE IF NOT EXISTS default"); - } - - protected List sql(String query, Object... args) { - List rows = spark.sql(String.format(query, args)).collectAsList(); - if (rows.size() < 1) { - return ImmutableList.of(); - } - - return rowsToJava(rows); - } - - protected List rowsToJava(List rows) { - return rows.stream().map(this::toJava).collect(Collectors.toList()); - } - - private Object[] toJava(Row row) { - return IntStream.range(0, row.size()) - .mapToObj( - pos -> { - if (row.isNullAt(pos)) { - return null; - } - - Object value = row.get(pos); - if (value instanceof Row) { - return toJava((Row) value); - } else if (value instanceof scala.collection.Seq) { - return row.getList(pos); - } else if (value instanceof scala.collection.Map) { - return row.getJavaMap(pos); - } else { - return value; - } - }) - .toArray(Object[]::new); } } diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 3f3d9e113c78..175863f1ffbb 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -82,7 +82,7 @@ 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 = "default"; + 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; @@ -182,6 +182,8 @@ public void before() throws IOException { newIcebergTableLocation = newIcebergTableFolder.toURI().toString(); externalDataFilesTableLocation = externalDataFilesTableFolder.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); @@ -240,6 +242,11 @@ public void after() { 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 DATABASE IF EXISTS %s", NAMESPACE)); } @Test @@ -252,6 +259,7 @@ public void testBasicSnapshotPartitioned() { checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, partitionedLocation); + spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -265,6 +273,7 @@ public void testBasicSnapshotUnpartitioned() { checkSnapshotIntegrity( unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); + spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -278,6 +287,7 @@ public void testSnapshotWithNewLocation() { checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); + spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -307,6 +317,7 @@ public void testSnapshotWithAdditionalProperties() { "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3", "test4", "test4"), unpartitionedLocation); + spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -324,6 +335,7 @@ public void testSnapshotTableWithExternalDataFiles() { externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation); checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); + spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } private void checkSnapshotIntegrity( From 6c4ab2c015a98231b921aeeba233fe2cafb8a7ee Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Thu, 19 Jan 2023 23:09:38 -0500 Subject: [PATCH 43/54] save getLength Call for AddFile and when RemoveFile contains size (#47) --- .../delta/BaseSnapshotDeltaLakeTableAction.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 8b96461d989a..69ef4d1d0322 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -270,15 +270,19 @@ private void commitDeltaVersionLogToIcebergTransaction( private DataFile buildDataFileFromAction(Action action, Table table) { PartitionSpec spec = table.spec(); String path; + long fileSize; + Long nullableFileSize; Map partitionValues; if (action instanceof AddFile) { AddFile addFile = (AddFile) action; path = addFile.getPath(); + nullableFileSize = addFile.getSize(); partitionValues = addFile.getPartitionValues(); } else if (action instanceof RemoveFile) { RemoveFile removeFile = (RemoveFile) action; path = removeFile.getPath(); + nullableFileSize = removeFile.getSize().orElse(null); partitionValues = removeFile.getPartitionValues(); } else { throw new ValidationException( @@ -293,6 +297,15 @@ private DataFile buildDataFileFromAction(Action action, Table table) { FileFormat format = determineFileFormatFromPath(fullFilePath); InputFile file = deltaLakeFileIO.newInputFile(fullFilePath); + + // If the file size is not specified, we need to read the file to get the file size + if (nullableFileSize != null) { + fileSize = nullableFileSize; + } else { + fileSize = file.getLength(); + } + + // get metrics from the file MetricsConfig metricsConfig = MetricsConfig.forTable(table); String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping nameMapping = @@ -308,7 +321,7 @@ private DataFile buildDataFileFromAction(Action action, Table table) { return DataFiles.builder(spec) .withPath(fullFilePath) .withFormat(format) - .withFileSizeInBytes(file.getLength()) + .withFileSizeInBytes(fileSize) .withMetrics(metrics) .withPartitionPath(partition) .build(); From dadd76aed170ffb3a38d5303c9cb75446b9206d1 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Sat, 21 Jan 2023 16:36:17 -0500 Subject: [PATCH 44/54] add null check for table.currentSnapshot() when querying the total number of data files migrated --- .../delta/BaseSnapshotDeltaLakeTableAction.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 69ef4d1d0322..fdb9c4491fbc 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -41,6 +41,7 @@ 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.TableProperties; @@ -172,13 +173,11 @@ public Result execute() { VersionLog versionLog = versionLogIterator.next(); commitDeltaVersionLogToIcebergTransaction(versionLog, icebergTransaction); } + Snapshot icebergSnapshot = icebergTransaction.table().currentSnapshot(); long totalDataFiles = - Long.parseLong( - icebergTransaction - .table() - .currentSnapshot() - .summary() - .get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); + icebergSnapshot != null + ? Long.parseLong(icebergSnapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)) + : 0; icebergTransaction.commitTransaction(); LOG.info( From 1cd36b97c90a47f400531a363c620f8253a5b73c Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Mon, 23 Jan 2023 22:48:10 -0500 Subject: [PATCH 45/54] Refactor iceberg-delta's integration test(#48) * use assertj for all tests * add null check for the spark integration method * use a method to generate the hardcode dataframe * drop iceberg table afterwards * add typetest table * test all delta lake types * test conversion of NullType * fix format issue * add a second dataframe * refactor the integration test * correctly decoded delta's path * fix wrong decoding * fix wrong decoding 2 --- ...akeToIcebergMigrationSparkIntegration.java | 10 + .../delta/TestSnapshotDeltaLakeTable.java | 284 ++++++++++-------- .../BaseSnapshotDeltaLakeTableAction.java | 15 +- .../delta/TestDeltaLakeTypeToType.java | 34 ++- 4 files changed, 205 insertions(+), 138 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java index 538dc708157c..5a3ad24a0e6b 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationSparkIntegration.java @@ -19,6 +19,7 @@ package org.apache.iceberg.delta; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.CatalogPlugin; @@ -40,6 +41,15 @@ private DeltaLakeToIcebergMigrationSparkIntegration() {} */ static SnapshotDeltaLakeTable snapshotDeltaLakeTable( SparkSession spark, String newTableIdentifier, String deltaTableLocation) { + Preconditions.checkArgument( + spark != null, "The SparkSession cannot be null, please provide a valid SparkSession"); + Preconditions.checkArgument( + newTableIdentifier != null, + "The table identifier cannot be null, please provide a valid table identifier for the new iceberg table"); + Preconditions.checkArgument( + deltaTableLocation != null, + "The delta lake table location cannot be null, please provide a valid location of the delta lake table to be snapshot"); + String ctx = "delta lake snapshot target"; CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); Spark3Util.CatalogAndIdentifier catalogAndIdent = diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 175863f1ffbb..24b76b0057dc 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -18,6 +18,10 @@ */ 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; @@ -30,23 +34,21 @@ 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.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.iceberg.spark.SparkCatalog; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.delta.catalog.DeltaCatalog; import org.assertj.core.api.Assertions; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -56,29 +58,6 @@ @RunWith(Parameterized.class) public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { - private static final String row1 = - "{\"name\":\"Michael\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," - + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ\",\"city\":\"Newark\"}}," - + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"]," - + "[\"Spark\",\"Java\"]],\"id\":1,\"magic_number\":1.123123123123}"; - private static final String row2 = - "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJos123123e\",\"state\":\"CA\"},{\"city\":\"Sand12312iago\",\"state\":\"CA\"}]," - + "\"address_nested\":{\"current\":{\"state\":\"N12Y\",\"city\":\"NewY1231ork\"}},\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"}," - + "\"secondProp\":{\"height\":\"6\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":2,\"magic_number\":2.123123123123}"; - private static final String row3 = - "{\"name\":\"Test\",\"addresses\":[{\"city\":\"SanJose\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," - + "\"properties\":{\"hair\":\"brown\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"6\"},\"subjects\":" - + "[[\"Java\",\"Scala\",\"C++\"],[\"Spark\",\"Java\"]],\"id\":3,\"magic_number\":3.123123123123}"; - private static final String row4 = - "{\"name\":\"John\",\"addresses\":[{\"city\":\"LA\",\"state\":\"CA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," - + "\"address_nested\":{\"current\":{\"state\":\"NY\",\"city\":\"NewYork\"},\"previous\":{\"state\":\"NJ123\"}}," - + "\"properties\":{\"hair\":\"b12rown\",\"eye\":\"bla3221ck\"},\"secondProp\":{\"height\":\"633\"},\"subjects\":" - + "[[\"Spark\",\"Java\"]],\"id\":4,\"magic_number\":4.123123123123}"; - private static final String row5 = - "{\"name\":\"Jonas\",\"addresses\":[{\"city\":\"Pittsburgh\",\"state\":\"PA\"},{\"city\":\"Sandiago\",\"state\":\"CA\"}]," - + "\"address_nested\":{\"current\":{\"state\":\"PA\",\"city\":\"Haha\"},\"previous\":{\"state\":\"NJ\"}}," - + "\"properties\":{\"hair\":\"black\",\"eye\":\"black\"},\"secondProp\":{\"height\":\"7\"},\"subjects\":[[\"Java\",\"Scala\",\"C++\"]," - + "[\"Spark\",\"Java\"]],\"id\":5,\"magic_number\":5.123123123123}"; private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; @@ -88,20 +67,30 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { 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, - SparkSessionCatalog.class.getName(), + SparkCatalog.class.getName(), ImmutableMap.of( "type", "hive", @@ -120,6 +109,7 @@ public static Object[][] parameters() { @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 config) { @@ -127,102 +117,41 @@ public TestSnapshotDeltaLakeTable( spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); } - /** - * The test hardcode a nested dataframe to test the snapshot feature. The schema of created - * dataframe is: - * - *

-   *  root
-   *  |-- address_nested: struct (nullable = true)
-   *  |    |-- current: struct (nullable = true)
-   *  |    |    |-- city: string (nullable = true)
-   *  |    |    |-- state: string (nullable = true)
-   *  |    |-- previous: struct (nullable = true)
-   *  |    |    |-- city: string (nullable = true)
-   *  |    |    |-- state: string (nullable = true)
-   *  |-- addresses: array (nullable = true)
-   *  |    |-- element: struct (containsNull = true)
-   *  |    |    |-- city: string (nullable = true)
-   *  |    |    |-- state: string (nullable = true)
-   *  |-- id: long (nullable = true)
-   *  |-- magic_number: double (nullable = true)
-   *  |-- name: string (nullable = true)
-   *  |-- properties: struct (nullable = true)
-   *  |    |-- eye: string (nullable = true)
-   *  |    |-- hair: string (nullable = true)
-   *  |-- secondProp: struct (nullable = true)
-   *  |    |-- height: string (nullable = true)
-   *  |-- subjects: array (nullable = true)
-   *  |    |-- element: array (containsNull = true)
-   *  |    |    |-- element: string (containsNull = true)
-   * 
- * - * The dataframe content is (by calling df.show()): - * - *
-   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
-   * |      address_nested|           addresses| id|  magic_number|   name|          properties|secondProp|            subjects|
-   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
-   * |{{NewYork, NY}, {...|[{SanJose, CA}, {...|  1|1.123123123123|Michael|      {black, brown}|       {6}|[[Java, Scala, C+...|
-   * |{{NewY1231ork, N1...|[{SanJos123123e, ...|  2|2.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
-   * |                null|[{SanJose, CA}, {...|  3|3.123123123123|   Test|      {black, brown}|       {6}|[[Java, Scala, C+...|
-   * |{{NewYork, NY}, {...|[{LA, CA}, {Sandi...|  4|4.123123123123|   John|{bla3221ck, b12rown}|     {633}|     [[Spark, Java]]|
-   * |{{Haha, PA}, {nul...|[{Pittsburgh, PA}...|  5|5.123123123123|  Jonas|      {black, black}|       {7}|[[Java, Scala, C+...|
-   * +--------------------+--------------------+---+--------------+-------+--------------------+----------+--------------------+
-   * 
- */ @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)); - // hard code the dataframe - List jsonList = Lists.newArrayList(); - jsonList.add(row1); - jsonList.add(row2); - jsonList.add(row3); - jsonList.add(row4); - jsonList.add(row5); - JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - SQLContext sqlContext = new SQLContext(javaSparkContext); - JavaRDD rdd = javaSparkContext.parallelize(jsonList); - Dataset df = sqlContext.read().json(rdd); + // generate the dataframe + Dataset nestedDataFrame = nestedDataFrame(); + Dataset typeTestDataFrame = typeTestDataFrame(); // write to delta tables - df.write() - .format("delta") - .mode(SaveMode.Append) - .partitionBy("id") - .option("path", partitionedLocation) - .saveAsTable(partitionedIdentifier); - - df.write() - .format("delta") - .mode(SaveMode.Append) - .option("path", unpartitionedLocation) - .saveAsTable(unpartitionedIdentifier); - - df.write() - .format("delta") - .mode(SaveMode.Append) - .option("path", externalDataFilesTableLocation) - .saveAsTable(externalDataFilesIdentifier); + 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"); @@ -245,13 +174,39 @@ public void after() { 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, "iceberg_table"); + String newTableIdentifier = destName(icebergCatalogName, snapshotPartitionedTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -259,12 +214,11 @@ public void testBasicSnapshotPartitioned() { checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, partitionedLocation); - spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test public void testBasicSnapshotUnpartitioned() { - String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_unpartitioned"); + String newTableIdentifier = destName(icebergCatalogName, snapshotUnpartitionedTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -273,12 +227,11 @@ public void testBasicSnapshotUnpartitioned() { checkSnapshotIntegrity( unpartitionedLocation, unpartitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); - spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test public void testSnapshotWithNewLocation() { - String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_new_location"); + String newTableIdentifier = destName(icebergCatalogName, snapshotNewTableLocationTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -287,7 +240,6 @@ public void testSnapshotWithNewLocation() { checkSnapshotIntegrity(partitionedLocation, partitionedIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); - spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -297,7 +249,7 @@ public void testSnapshotWithAdditionalProperties() { "ALTER TABLE " + unpartitionedIdentifier + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_additional_properties"); + String newTableIdentifier = destName(icebergCatalogName, snapshotAdditionalPropertiesTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -317,7 +269,6 @@ public void testSnapshotWithAdditionalProperties() { "foo", "bar", "test0", "test0", "test1", "test1", "test2", "test2", "test3", "test3", "test4", "test4"), unpartitionedLocation); - spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); } @Test @@ -326,7 +277,7 @@ public void testSnapshotTableWithExternalDataFiles() { // are not at the same location as the table. addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_table_external_data_files"); + String newTableIdentifier = destName(icebergCatalogName, snapshotExternalDataFilesTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, externalDataFilesTableLocation) @@ -335,7 +286,22 @@ public void testSnapshotTableWithExternalDataFiles() { externalDataFilesTableLocation, externalDataFilesIdentifier, newTableIdentifier, result); checkIcebergTableLocation(newTableIdentifier, externalDataFilesTableLocation); checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); - spark.sql(String.format("DROP TABLE IF EXISTS %s", newTableIdentifier)); + } + + @Test + public void testSnapshotSupportedTypes() { + String newTableIdentifier = destName(icebergCatalogName, snapshotTypeTestTableName); + SnapshotDeltaLakeTable.Result result = + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( + spark, newTableIdentifier, typeTestTableLocation) + .tableProperty(TableProperties.PARQUET_VECTORIZATION_ENABLED, "false") + .execute(); + checkSnapshotIntegrity(typeTestTableLocation, typeTestIdentifier, newTableIdentifier, result); + checkIcebergTableLocation(newTableIdentifier, typeTestTableLocation); + checkIcebergTableProperties( + newTableIdentifier, + ImmutableMap.of(TableProperties.PARQUET_VECTORIZATION_ENABLED, "false"), + typeTestTableLocation); } private void checkSnapshotIntegrity( @@ -350,24 +316,16 @@ private void checkSnapshotIntegrity( List icebergTableContents = spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList(); - Assert.assertEquals( - "The original table and the transformed one should have the same size", - deltaTableContents.size(), - icebergTableContents.size()); - Assert.assertEquals( - "The number of files in the delta table should be the same as the number of files in the snapshot iceberg table", - deltaLog.update().getAllFiles().size(), - snapshotReport.snapshotDataFilesCount()); + Assertions.assertThat(deltaTableContents).hasSize(icebergTableContents.size()); + Assertions.assertThat(deltaLog.update().getAllFiles()) + .hasSize((int) snapshotReport.snapshotDataFilesCount()); Assertions.assertThat(icebergTableContents).containsAll(deltaTableContents); Assertions.assertThat(deltaTableContents).containsAll(icebergTableContents); } private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) { Table icebergTable = getIcebergTable(icebergTableIdentifier); - Assert.assertEquals( - "The iceberg table should have the expected location", - expectedLocation, - icebergTable.location()); + Assertions.assertThat(icebergTable.location()).isEqualTo(expectedLocation); } private void checkIcebergTableProperties( @@ -402,6 +360,7 @@ private void checkDataFilePathsIntegrity( .addedDataFiles(icebergTable.io()) .forEach( dataFile -> { + Assertions.assertThat(URI.create(dataFile.path().toString()).isAbsolute()).isTrue(); Assertions.assertThat(deltaTableDataFilePaths).contains(dataFile.path().toString()); }); } @@ -459,10 +418,79 @@ private void addExternalDatafiles( private static String getFullFilePath(String path, String tableRoot) { URI dataFileUri = URI.create(path); - if (dataFileUri.isAbsolute()) { - return path; + try { + String decodedPath = new URLCodec().decode(path); + if (dataFileUri.isAbsolute()) { + return decodedPath; + } else { + return tableRoot + File.separator + decodedPath; + } + } catch (DecoderException e) { + throw new IllegalArgumentException(String.format("Cannot decode path %s", path), e); + } + } + + private Dataset typeTestDataFrame() { + return spark + .range(0, 5, 1, 5) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("dateCol", date_add(current_date(), 1)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(timestampCol AS STRING)")) + .withColumn("booleanCol", expr("longCol > 5")) + .withColumn("binaryCol", expr("CAST(longCol AS BINARY)")) + .withColumn("byteCol", expr("CAST(longCol AS BYTE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(10, 2))")) + .withColumn("shortCol", expr("CAST(longCol AS SHORT)")) + .withColumn("mapCol", expr("MAP(longCol, decimalCol)")) + .withColumn("arrayCol", expr("ARRAY(longCol)")) + .withColumn("structCol", expr("STRUCT(mapCol, arrayCol)")); + } + + private Dataset nestedDataFrame() { + return spark + .range(0, 5, 1, 5) + .withColumn("longCol", expr("id")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(10, 2))")) + .withColumn("magic_number", expr("rand(5) * 100")) + .withColumn("dateCol", date_add(current_date(), 1)) + .withColumn("dateString", expr("CAST(dateCol AS STRING)")) + .withColumn("random1", expr("CAST(rand(5) * 100 as LONG)")) + .withColumn("random2", expr("CAST(rand(51) * 100 as LONG)")) + .withColumn("random3", expr("CAST(rand(511) * 100 as LONG)")) + .withColumn("random4", expr("CAST(rand(15) * 100 as LONG)")) + .withColumn("random5", expr("CAST(rand(115) * 100 as LONG)")) + .withColumn("innerStruct1", expr("STRUCT(random1, random2)")) + .withColumn("innerStruct2", expr("STRUCT(random3, random4)")) + .withColumn("structCol1", expr("STRUCT(innerStruct1, innerStruct2)")) + .withColumn( + "innerStruct3", + expr("STRUCT(SHA1(CAST(random5 AS BINARY)), SHA1(CAST(random1 AS BINARY)))")) + .withColumn( + "structCol2", + expr( + "STRUCT(innerStruct3, STRUCT(SHA1(CAST(random2 AS BINARY)), SHA1(CAST(random3 AS BINARY))))")) + .withColumn("arrayCol", expr("ARRAY(random1, random2, random3, random4, random5)")) + .withColumn("mapCol1", expr("MAP(structCol1, structCol2)")) + .withColumn("mapCol2", expr("MAP(longCol, dateString)")) + .withColumn("mapCol3", expr("MAP(dateCol, arrayCol)")) + .withColumn("structCol3", expr("STRUCT(structCol2, mapCol3, arrayCol)")); + } + + private void writeDeltaTable( + Dataset df, String identifier, String path, String partitionColumn) { + if (partitionColumn != null) { + df.write() + .format("delta") + .mode(SaveMode.Append) + .option("path", path) + .partitionBy(partitionColumn) + .saveAsTable(identifier); } else { - return tableRoot + File.separator + path; + df.write().format("delta").mode(SaveMode.Append).option("path", path).saveAsTable(identifier); } } } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index fdb9c4491fbc..786a357ed658 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -29,6 +29,8 @@ 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.hadoop.conf.Configuration; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; @@ -373,10 +375,15 @@ private Map destTableProperties( */ private static String getFullFilePath(String path, String tableRoot) { URI dataFileUri = URI.create(path); - if (dataFileUri.isAbsolute()) { - return path; - } else { - return tableRoot + File.separator + path; + try { + String decodedPath = new URLCodec().decode(path); + if (dataFileUri.isAbsolute()) { + return decodedPath; + } else { + return tableRoot + File.separator + decodedPath; + } + } catch (DecoderException e) { + throw new IllegalArgumentException(String.format("Cannot decode path %s", path), e); } } } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index fb7166fca223..996e28117862 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -24,9 +24,11 @@ import io.delta.standalone.types.DoubleType; import io.delta.standalone.types.LongType; import io.delta.standalone.types.MapType; +import io.delta.standalone.types.NullType; import io.delta.standalone.types.StringType; import io.delta.standalone.types.StructType; import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; @@ -35,18 +37,15 @@ public class TestDeltaLakeTypeToType { private static final String optionalBooleanType = "testNullableBoolType"; - private static final String requiredBinaryType = "testRequiredBinaryType"; - private static final String doubleArrayType = "testNullableArrayType"; - private static final String innerAtomicSchema = "testInnerAtomicSchema"; - private static final String stringLongMapType = "testStringLongMap"; - + private static final String nullType = "testNullType"; private StructType deltaAtomicSchema; - private StructType deltaNestedSchema; + private StructType deltaShallowNullTypeSchema; + private StructType deltaNullTypeSchema; @Before public void constructDeltaLakeSchema() { @@ -59,6 +58,12 @@ public void constructDeltaLakeSchema() { .add(innerAtomicSchema, deltaAtomicSchema) .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false); + deltaNullTypeSchema = + new StructType() + .add(innerAtomicSchema, deltaAtomicSchema) + .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) + .add(stringLongMapType, new MapType(new NullType(), new LongType(), false), false); + deltaShallowNullTypeSchema = new StructType().add(nullType, new NullType(), false); } @Test @@ -118,4 +123,21 @@ public void testNestedTypeConversion() { convertedSchema.findType(doubleArrayType).asListType().isElementOptional()) .isTrue(); } + + @Test + public void testNullTypeConversion() { + Assertions.assertThatThrownBy( + () -> + DeltaLakeDataTypeVisitor.visit( + deltaNullTypeSchema, new DeltaLakeTypeToType(deltaNullTypeSchema))) + .isInstanceOf(ValidationException.class) + .hasMessage(String.format("Not a supported type: %s", new NullType().getCatalogString())); + Assertions.assertThatThrownBy( + () -> + DeltaLakeDataTypeVisitor.visit( + deltaShallowNullTypeSchema, + new DeltaLakeTypeToType(deltaShallowNullTypeSchema))) + .isInstanceOf(ValidationException.class) + .hasMessage(String.format("Not a supported type: %s", new NullType().getCatalogString())); + } } From 4463f30f16379cb75ba7f49eb16ba00a4c8b71a5 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Wed, 25 Jan 2023 18:23:24 -0500 Subject: [PATCH 46/54] Adapt for delta.logRetentionDuration (#49) * remove a redundant map collector in commitDeltaVersionLogToIcebergTransaction * get the earliest possible version rather than hard code from 0 * add unit test to check if table exists * refactor action extracted from the versionlog * fix format issue * move non-share table write operation to the test itself, instead of in before() * fix type --- .../delta/TestSnapshotDeltaLakeTable.java | 12 ++++--- .../BaseSnapshotDeltaLakeTableAction.java | 36 +++++++++++++------ .../TestBaseSnapshotDeltaLakeTableAction.java | 14 ++++++++ 3 files changed, 46 insertions(+), 16 deletions(-) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 24b76b0057dc..72352e71358a 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -84,6 +84,8 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private String newIcebergTableLocation; private String externalDataFilesTableLocation; private String typeTestTableLocation; + private Dataset typeTestDataFrame; + private Dataset nestedDataFrame; @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") public static Object[][] parameters() { @@ -143,15 +145,12 @@ public void before() throws IOException { spark.sql(String.format("DROP TABLE IF EXISTS %s", typeTestIdentifier)); // generate the dataframe - Dataset nestedDataFrame = nestedDataFrame(); - Dataset typeTestDataFrame = typeTestDataFrame(); + nestedDataFrame = nestedDataFrame(); + 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"); @@ -273,6 +272,8 @@ public void testSnapshotWithAdditionalProperties() { @Test public void testSnapshotTableWithExternalDataFiles() { + writeDeltaTable( + nestedDataFrame, externalDataFilesIdentifier, externalDataFilesTableLocation, null); // 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); @@ -290,6 +291,7 @@ public void testSnapshotTableWithExternalDataFiles() { @Test public void testSnapshotSupportedTypes() { + writeDeltaTable(typeTestDataFrame, typeTestIdentifier, typeTestTableLocation, "stringCol"); String newTableIdentifier = destName(icebergCatalogName, snapshotTypeTestTableName); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 786a357ed658..29cfc45a9b21 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -61,7 +61,6 @@ import org.apache.iceberg.parquet.ParquetUtil; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; 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.types.Type; import org.slf4j.Logger; @@ -90,6 +89,7 @@ class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { private TableIdentifier newTableIdentifier; private String newTableLocation; private HadoopFileIO deltaLakeFileIO; + private long deltaStartVersion; /** * Snapshot a delta lake table to be an iceberg table. The action will read the delta lake table's @@ -139,6 +139,8 @@ public SnapshotDeltaLakeTable icebergCatalog(Catalog catalog) { public SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf) { this.deltaLog = DeltaLog.forTable(conf, deltaTableLocation); this.deltaLakeFileIO = new HadoopFileIO(conf); + // get the earliest version available in the delta lake table + this.deltaStartVersion = deltaLog.getVersionAtOrAfterTimestamp(0L); return this; } @@ -150,6 +152,10 @@ public Result execute() { Preconditions.checkArgument( deltaLog != null && deltaLakeFileIO != null, "Make sure to configure the action with a valid deltaLakeConfiguration"); + Preconditions.checkArgument( + deltaLog.tableExists(), + "Delta lake table does not exist at the given location: %s", + deltaTableLocation); io.delta.standalone.Snapshot updatedSnapshot = deltaLog.update(); Schema schema = convertDeltaLakeSchema(updatedSnapshot.getMetadata().getSchema()); PartitionSpec partitionSpec = getPartitionSpecFromDeltaSnapshot(schema); @@ -169,8 +175,8 @@ public Result execute() { .commit(); Iterator versionLogIterator = deltaLog.getChanges( - 0, // retrieve actions starting from the initial version - false); // not throw exception when data loss detected + deltaStartVersion, false // not throw exception when data loss detected + ); while (versionLogIterator.hasNext()) { VersionLog versionLog = versionLogIterator.next(); commitDeltaVersionLogToIcebergTransaction(versionLog, icebergTransaction); @@ -227,17 +233,25 @@ private PartitionSpec getPartitionSpecFromDeltaSnapshot(Schema schema) { */ private void commitDeltaVersionLogToIcebergTransaction( VersionLog versionLog, Transaction transaction) { - List actions = versionLog.getActions(); - - // Create a map of Delta Lake Action (AddFile, RemoveFile, etc.) --> List - Map> deltaLakeActionMap = - actions.stream() - .filter(action -> action instanceof AddFile || action instanceof RemoveFile) - .collect(Collectors.groupingBy(a -> a.getClass().getSimpleName())); + List dataFileActions; + if (versionLog.getVersion() == deltaStartVersion) { + // The first version log is a special case, since it contains the initial table state. + // we need to get all dataFiles from the corresponding delta snapshot to construct the table. + dataFileActions = + deltaLog.getSnapshotForVersionAsOf(deltaStartVersion).getAllFiles().stream() + .map(addFile -> (Action) addFile) + .collect(Collectors.toList()); + } else { + // Only need actions related to data change: AddFile and RemoveFile + dataFileActions = + versionLog.getActions().stream() + .filter(action -> action instanceof AddFile || action instanceof RemoveFile) + .collect(Collectors.toList()); + } List filesToAdd = Lists.newArrayList(); List filesToRemove = Lists.newArrayList(); - for (Action action : Iterables.concat(deltaLakeActionMap.values())) { + for (Action action : dataFileActions) { DataFile dataFile = buildDataFileFromAction(action, transaction.table()); if (action instanceof AddFile) { filesToAdd.add(dataFile); diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java index 941bc4a990b3..41261d4ab975 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -87,6 +87,20 @@ public void testRequiredDeltaLakeConfiguration() { .hasMessage("Make sure to configure the action with a valid deltaLakeConfiguration"); } + @Test + public void testDeltaTableNotExist() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + Assertions.assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Delta lake table does not exist at the given location: %s", sourceTableLocation); + } + private static class TestCatalog extends BaseMetastoreCatalog { TestCatalog() {} From d3ccc8644ca7ae52301692514d696dd94e585432 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Wed, 25 Jan 2023 22:02:11 -0500 Subject: [PATCH 47/54] fix comment and format issue --- .../iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 29cfc45a9b21..c6607bbd1bf2 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -173,6 +173,7 @@ public Result execute() { TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(MappingUtil.create(icebergTransaction.table().schema()))) .commit(); + Iterator versionLogIterator = deltaLog.getChanges( deltaStartVersion, false // not throw exception when data loss detected @@ -181,6 +182,7 @@ public Result execute() { VersionLog versionLog = versionLogIterator.next(); commitDeltaVersionLogToIcebergTransaction(versionLog, icebergTransaction); } + Snapshot icebergSnapshot = icebergTransaction.table().currentSnapshot(); long totalDataFiles = icebergSnapshot != null @@ -235,8 +237,8 @@ private void commitDeltaVersionLogToIcebergTransaction( VersionLog versionLog, Transaction transaction) { List dataFileActions; if (versionLog.getVersion() == deltaStartVersion) { - // The first version log is a special case, since it contains the initial table state. - // we need to get all dataFiles from the corresponding delta snapshot to construct the table. + // The first version is a special case, since it represents the initial table state. + // Need to get all dataFiles from the corresponding delta snapshot to construct the table. dataFileActions = deltaLog.getSnapshotForVersionAsOf(deltaStartVersion).getAllFiles().stream() .map(addFile -> (Action) addFile) @@ -313,7 +315,7 @@ private DataFile buildDataFileFromAction(Action action, Table table) { FileFormat format = determineFileFormatFromPath(fullFilePath); InputFile file = deltaLakeFileIO.newInputFile(fullFilePath); - // If the file size is not specified, we need to read the file to get the file size + // If the file size is not specified, the size should be read from the file if (nullableFileSize != null) { fileSize = nullableFileSize; } else { From 1affcb32c498b58a8f37451a7a4f64076fff0e10 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Mon, 6 Feb 2023 17:40:25 -0500 Subject: [PATCH 48/54] remove support for avro, orc since it can allow use to get rid of a dependency --- build.gradle | 1 - .../delta/BaseSnapshotDeltaLakeTableAction.java | 15 +++------------ 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/build.gradle b/build.gradle index 508310ea0703..32231049865b 100644 --- a/build.gradle +++ b/build.gradle @@ -456,7 +456,6 @@ project(':iceberg-delta-lake') { implementation project(':iceberg-common') implementation project(':iceberg-core') implementation project(':iceberg-parquet') - implementation project(':iceberg-orc') implementation "com.fasterxml.jackson.core:jackson-databind" compileOnly "io.delta:delta-standalone_${scalaVersion}" diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index c6607bbd1bf2..12a4dfa2bde6 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -48,7 +48,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Transaction; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ValidationException; @@ -57,7 +56,6 @@ import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.OrcMetrics; import org.apache.iceberg.parquet.ParquetUtil; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -358,17 +356,10 @@ private FileFormat determineFileFormatFromPath(String path) { private Metrics getMetricsForFile( InputFile file, FileFormat format, MetricsConfig metricsSpec, NameMapping mapping) { - switch (format) { - case AVRO: - long rowCount = Avro.rowCount(file); - return new Metrics(rowCount, null, null, null, null); - case PARQUET: - return ParquetUtil.fileMetrics(file, metricsSpec, mapping); - case ORC: - return OrcMetrics.fromInputFile(file, metricsSpec, mapping); - default: - throw new ValidationException("Cannot get metrics from file format: %s", format); + if (format == FileFormat.PARQUET) { + return ParquetUtil.fileMetrics(file, metricsSpec, mapping); } + throw new ValidationException("Cannot get metrics from file format: %s", format); } private Map destTableProperties( From 098a3a2ede4ba0ff97cc3956216abd8717555837 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Mon, 6 Feb 2023 17:57:18 -0500 Subject: [PATCH 49/54] using resolvingFileIO instead --- .../iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 12a4dfa2bde6..457b34353ada 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -51,8 +51,8 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.ResolvingFileIO; import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; @@ -86,7 +86,7 @@ class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { private final String deltaTableLocation; private TableIdentifier newTableIdentifier; private String newTableLocation; - private HadoopFileIO deltaLakeFileIO; + private ResolvingFileIO deltaLakeFileIO; private long deltaStartVersion; /** @@ -136,7 +136,9 @@ public SnapshotDeltaLakeTable icebergCatalog(Catalog catalog) { @Override public SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf) { this.deltaLog = DeltaLog.forTable(conf, deltaTableLocation); - this.deltaLakeFileIO = new HadoopFileIO(conf); + this.deltaLakeFileIO = new ResolvingFileIO(); + this.deltaLakeFileIO.initialize(ImmutableMap.of()); + this.deltaLakeFileIO.setConf(conf); // get the earliest version available in the delta lake table this.deltaStartVersion = deltaLog.getVersionAtOrAfterTimestamp(0L); return this; From a98461a9f0c46155a6bbcdececc1acc2da5e7c65 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 7 Feb 2023 00:45:00 -0500 Subject: [PATCH 50/54] rollback to hadoopFileIO --- .../iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 457b34353ada..12a4dfa2bde6 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -51,8 +51,8 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.io.ResolvingFileIO; import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; @@ -86,7 +86,7 @@ class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { private final String deltaTableLocation; private TableIdentifier newTableIdentifier; private String newTableLocation; - private ResolvingFileIO deltaLakeFileIO; + private HadoopFileIO deltaLakeFileIO; private long deltaStartVersion; /** @@ -136,9 +136,7 @@ public SnapshotDeltaLakeTable icebergCatalog(Catalog catalog) { @Override public SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf) { this.deltaLog = DeltaLog.forTable(conf, deltaTableLocation); - this.deltaLakeFileIO = new ResolvingFileIO(); - this.deltaLakeFileIO.initialize(ImmutableMap.of()); - this.deltaLakeFileIO.setConf(conf); + this.deltaLakeFileIO = new HadoopFileIO(conf); // get the earliest version available in the delta lake table this.deltaStartVersion = deltaLog.getVersionAtOrAfterTimestamp(0L); return this; From fe6da1799d6a7f712bbbcbe81240ef6a23297def Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 7 Feb 2023 15:12:20 -0500 Subject: [PATCH 51/54] add test for array of structs --- .../delta/TestSnapshotDeltaLakeTable.java | 1 + .../delta/TestDeltaLakeTypeToType.java | 42 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 72352e71358a..c79129e0af5e 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -476,6 +476,7 @@ private Dataset nestedDataFrame() { expr( "STRUCT(innerStruct3, STRUCT(SHA1(CAST(random2 AS BINARY)), SHA1(CAST(random3 AS BINARY))))")) .withColumn("arrayCol", expr("ARRAY(random1, random2, random3, random4, random5)")) + .withColumn("arrayStructCol", expr("ARRAY(innerStruct1, innerStruct1, innerStruct1)")) .withColumn("mapCol1", expr("MAP(structCol1, structCol2)")) .withColumn("mapCol2", expr("MAP(longCol, dateString)")) .withColumn("mapCol3", expr("MAP(dateCol, arrayCol)")) diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 996e28117862..a0faaebc850d 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -39,6 +39,7 @@ public class TestDeltaLakeTypeToType { private static final String optionalBooleanType = "testNullableBoolType"; private static final String requiredBinaryType = "testRequiredBinaryType"; private static final String doubleArrayType = "testNullableArrayType"; + private static final String structArrayType = "testStructArrayType"; private static final String innerAtomicSchema = "testInnerAtomicSchema"; private static final String stringLongMapType = "testStringLongMap"; private static final String nullType = "testNullType"; @@ -57,6 +58,7 @@ public void constructDeltaLakeSchema() { new StructType() .add(innerAtomicSchema, deltaAtomicSchema) .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) + .add(structArrayType, new ArrayType(deltaAtomicSchema, true), false) .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false); deltaNullTypeSchema = new StructType() @@ -122,6 +124,46 @@ public void testNestedTypeConversion() { Assertions.assertThat( convertedSchema.findType(doubleArrayType).asListType().isElementOptional()) .isTrue(); + Assertions.assertThat(convertedSchema.findType(structArrayType)) + .isInstanceOf(Types.ListType.class); + Assertions.assertThat(convertedSchema.findField(structArrayType).isRequired()).isTrue(); + Assertions.assertThat( + convertedSchema.findType(structArrayType).asListType().isElementOptional()) + .isTrue(); + Assertions.assertThat( + convertedSchema + .findType(structArrayType) + .asListType() + .elementType() + .asStructType() + .fieldType(optionalBooleanType)) + .isInstanceOf(Types.BooleanType.class); + Assertions.assertThat( + convertedSchema + .findType(structArrayType) + .asListType() + .elementType() + .asStructType() + .field(optionalBooleanType) + .isOptional()) + .isTrue(); + Assertions.assertThat( + convertedSchema + .findType(structArrayType) + .asListType() + .elementType() + .asStructType() + .fieldType(requiredBinaryType)) + .isInstanceOf(Types.BinaryType.class); + Assertions.assertThat( + convertedSchema + .findType(structArrayType) + .asListType() + .elementType() + .asStructType() + .field(requiredBinaryType) + .isRequired()) + .isTrue(); } @Test From 8e9a3e26923d0db74f1d2d7efb3ab1a2eb415c9a Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 7 Feb 2023 15:21:00 -0500 Subject: [PATCH 52/54] use Do not Support instead of cannot determine, remove support for avro and orc --- .../iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 12a4dfa2bde6..d4c3763bd3fc 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -77,8 +77,6 @@ class BaseSnapshotDeltaLakeTableAction implements SnapshotDeltaLakeTable { private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; private static final String PARQUET_SUFFIX = ".parquet"; - private static final String AVRO_SUFFIX = ".avro"; - private static final String ORC_SUFFIX = ".orc"; private final ImmutableMap.Builder additionalPropertiesBuilder = ImmutableMap.builder(); private DeltaLog deltaLog; @@ -345,12 +343,8 @@ private DataFile buildDataFileFromAction(Action action, Table table) { private FileFormat determineFileFormatFromPath(String path) { if (path.endsWith(PARQUET_SUFFIX)) { return FileFormat.PARQUET; - } else if (path.endsWith(AVRO_SUFFIX)) { - return FileFormat.AVRO; - } else if (path.endsWith(ORC_SUFFIX)) { - return FileFormat.ORC; } else { - throw new ValidationException("Cannot determine file format from path %s", path); + throw new ValidationException("Do not support file format from path %s", path); } } From 24405e0a7a8d1c6c8a0b2a057811d035679549cb Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 7 Feb 2023 15:28:09 -0500 Subject: [PATCH 53/54] nit fix --- .../java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index a0faaebc850d..29a5c63c3d22 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -130,6 +130,8 @@ public void testNestedTypeConversion() { Assertions.assertThat( convertedSchema.findType(structArrayType).asListType().isElementOptional()) .isTrue(); + Assertions.assertThat(convertedSchema.findType(structArrayType).asListType().elementType()) + .isInstanceOf(Types.StructType.class); Assertions.assertThat( convertedSchema .findType(structArrayType) From c5a6186a7bb67dabcbf5fbb5f2efdc1e8ffa2566 Mon Sep 17 00:00:00 2001 From: Rushan Jiang Date: Tue, 7 Feb 2023 15:39:15 -0500 Subject: [PATCH 54/54] error message nit fix --- .../apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index d4c3763bd3fc..3ab561de9a58 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -344,7 +344,7 @@ private FileFormat determineFileFormatFromPath(String path) { if (path.endsWith(PARQUET_SUFFIX)) { return FileFormat.PARQUET; } else { - throw new ValidationException("Do not support file format from path %s", path); + throw new ValidationException("Do not support file format in path %s", path); } }