diff --git a/build.gradle b/build.gradle index 4cf6cf2ad9d7..21155ed8150c 100644 --- a/build.gradle +++ b/build.gradle @@ -638,6 +638,8 @@ project(':iceberg-delta-lake') { compileOnly libs.immutables.value compileOnly "io.delta:delta-standalone_${scalaVersion}:${libs.versions.delta.standalone.get()}" + compileOnly "io.delta:delta-kernel-api:4.0.0" + compileOnly "io.delta:delta-kernel-defaults:4.0.0" compileOnly(libs.hadoop3.common) { exclude group: 'org.apache.avro', module: 'avro' 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 5a3ad24a0e6b..e1959844f88e 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 @@ -60,4 +60,35 @@ static SnapshotDeltaLakeTable snapshotDeltaLakeTable( .deltaLakeConfiguration(spark.sessionState().newHadoopConf()) .icebergCatalog(Spark3Util.loadIcebergCatalog(spark, catalogAndIdent.catalog().name())); } + + /** + * 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 + * 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 snapshotDeltaLakeKernelTable( + 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 = + Spark3Util.catalogAndIdentifier(ctx, spark, newTableIdentifier, defaultCatalog); + return new BaseSnapshotDeltaLakeKernelTableAction(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/TestSnapshotDeltaLakeKernelTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeKernelTable.java new file mode 100644 index 000000000000..034632c9b59e --- /dev/null +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeKernelTable.java @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.delta.kernel.defaults.engine.DefaultEngine; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +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.SparkCatalog; +import org.apache.iceberg.util.LocationUtil; +import org.apache.spark.sql.DataFrameWriter; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.delta.catalog.DeltaCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestSnapshotDeltaLakeKernelTable extends SparkDeltaLakeSnapshotTestBase { + private static final String NAMESPACE = "delta_conversion_test_ns"; + private static final String DEFAULT_SPARK_CATALOG = "spark_catalog"; + private static final String ICEBERG_CATALOG_NAME = "iceberg_hive"; + private static final Map BASE_SPARK_CONFIG = + 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 Dataset genericDataFrame; + + @TempDir private File sourceLocation; + @TempDir private File destinationLocation; + + public TestSnapshotDeltaLakeKernelTable() { + super(ICEBERG_CATALOG_NAME, SparkCatalog.class.getName(), BASE_SPARK_CONFIG); + spark.conf().set("spark.sql.catalog." + DEFAULT_SPARK_CATALOG, DeltaCatalog.class.getName()); + } + + @BeforeAll + public static void beforeClass() { + spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", NAMESPACE)); + StructType schema = + new StructType( + new StructField[] { + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("created_at", DataTypes.TimestampType, true, Metadata.empty()), + new StructField("event_name", DataTypes.StringType, true, Metadata.empty()), + new StructField("is_active", DataTypes.BooleanType, true, Metadata.empty()), + new StructField("price", DataTypes.DoubleType, true, Metadata.empty()) + }); + List data = + Arrays.asList( + RowFactory.create(1, Timestamp.valueOf("2025-01-01 10:00:00"), "Signup", true, 0.00), + RowFactory.create( + 2, Timestamp.valueOf("2025-01-02 14:30:00"), "Purchase", true, 199.99), + RowFactory.create( + 3, Timestamp.valueOf("2025-01-03 09:15:00"), "Deactivation", false, 0.00), + RowFactory.create(4, Timestamp.valueOf("2025-01-03 09:16:00"), "Refund", false, 0.00), + RowFactory.create(5, Timestamp.valueOf("2025-01-03 09:17:00"), "Refund", false, 0.00)); + genericDataFrame = spark.createDataFrame(data, schema); + } + + @AfterAll + public static void afterClass() { + spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", NAMESPACE)); + } + + @Test + public void testBasicPartitionedInsertsOnly() { + String sourceTable = toFullTableName(DEFAULT_SPARK_CATALOG, "partitioned_table"); + String sourceTableLocation = sourceLocation.toURI().toString(); + + writeDeltaTable(genericDataFrame, sourceTable, sourceTableLocation, "id"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (10, current_date(), null, null, null);"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (11, current_date(), null, null, null);"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (12, current_date(), null, null, null);"); + + String newTableIdentifier = toFullTableName(ICEBERG_CATALOG_NAME, "iceberg_partitioned_table"); + + // Act + SnapshotDeltaLakeTable conversionAction = + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeKernelTable( + spark, newTableIdentifier, sourceTableLocation); + SnapshotDeltaLakeTable.Result result = conversionAction.execute(); + + // Assert + checkLatestSnapshotIntegrity(sourceTable, newTableIdentifier); + checkTagContentAndOrder(sourceTable, sourceTableLocation, newTableIdentifier, 0); + checkIcebergTableLocation(newTableIdentifier, sourceTableLocation); + } + + @Test + public void testInsertUpdateDeleteSqls() { + String sourceTable = toFullTableName(DEFAULT_SPARK_CATALOG, "crud_table"); + String sourceTableLocation = sourceLocation.toURI().toString(); + + writeDeltaTable(genericDataFrame, sourceTable, sourceTableLocation, "id"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (10, current_date(), null, null, null);"); + spark.sql("DELETE FROM " + sourceTable + " WHERE id=3;"); + spark.sql("UPDATE " + sourceTable + " SET id=3 WHERE id=1;"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (11, current_date(), null, null, null);"); + + String newTableIdentifier = toFullTableName(ICEBERG_CATALOG_NAME, "iceberg_crud_table"); + + // Act + SnapshotDeltaLakeTable conversionAction = + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeKernelTable( + spark, newTableIdentifier, sourceTableLocation); + SnapshotDeltaLakeTable.Result result = conversionAction.execute(); + + // Assert + checkLatestSnapshotIntegrity(sourceTable, newTableIdentifier); + checkTagContentAndOrder(sourceTable, sourceTableLocation, newTableIdentifier, 0); + checkIcebergTableLocation(newTableIdentifier, sourceTableLocation); + } + + @Test + public void testConversionAfterVacuum() throws IOException { + String sourceTable = toFullTableName(DEFAULT_SPARK_CATALOG, "vacuumed_table"); + String sourceTableLocation = sourceLocation.toURI().toString(); + + writeDeltaTable(genericDataFrame, sourceTable, sourceTableLocation, "id"); + for (int i = 0; i < 5; i++) { + spark.sql( + "UPDATE " + + sourceTable + + " SET price=" + + ThreadLocalRandom.current().nextDouble(1000) + + " where id=" + + i + + ";"); + } + spark.sql("UPDATE " + sourceTable + " SET created_at=current_date() ;"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (10, current_date(), null, null, null);"); + spark.sql("INSERT INTO " + sourceTable + " VALUES (11, current_date(), null, null, null);"); + spark.sql("DELETE FROM " + sourceTable + " WHERE id>=10;"); + spark.sql("VACUUM " + sourceTable + " RETAIN 0 HOURS"); + spark.sql( + "INSERT INTO " + sourceTable + " VALUES (12, current_date(), 'after_vacuum', null, null);"); + spark.sql("UPDATE " + sourceTable + " SET id=13 WHERE id=5;"); + + // Checkpoint generated. Simulate logs clean-up + assertThat(deleteDeltaLogFile("00000000000000000000.json")).isTrue(); + assertThat(deleteDeltaLogFile("00000000000000000001.json")).isTrue(); + assertThat(deleteDeltaLogFile("00000000000000000002.json")).isTrue(); + + String newTableIdentifier = toFullTableName(ICEBERG_CATALOG_NAME, "iceberg_vacuumed_table"); + + // Act + SnapshotDeltaLakeTable conversionAction = + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeKernelTable( + spark, newTableIdentifier, sourceTableLocation); + SnapshotDeltaLakeTable.Result result = conversionAction.execute(); + + // Assert + checkLatestSnapshotIntegrity(sourceTable, newTableIdentifier); + checkTagContentAndOrder(sourceTable, sourceTableLocation, newTableIdentifier, 10); + checkIcebergTableLocation(newTableIdentifier, sourceTableLocation); + } + + @Test + public void testConversionWithDeletionVectors() { + String sourceTable = toFullTableName(DEFAULT_SPARK_CATALOG, "dv_table"); + String sourceTableLocation = sourceLocation.toURI().toString(); + + spark.sql(String.format("DROP TABLE IF EXISTS %s", sourceTable)); + + // 10k records to force usage of DVs in Update operation + Dataset dvDf = + spark + .range(10000) + .selectExpr( + "CAST(id AS int) AS id", + "current_timestamp() AS created_at", + "CAST(id AS string) AS event_name", + "CAST(id % 2 == 0 AS boolean) AS is_active", + "CAST(id AS double) AS price"); + + dvDf.write() + .format("delta") + .mode(SaveMode.Append) + .option("path", sourceTableLocation) + .option("delta.enableDeletionVectors", "true") + .option("delta.enableInCommitTimestamps", "true") + .option("delta.enableRowTracking", "true") + .saveAsTable(sourceTable); + + spark.sql("UPDATE " + sourceTable + " SET id=-1 WHERE id=1 OR id=2;"); + + String newTableIdentifier = toFullTableName(ICEBERG_CATALOG_NAME, "iceberg_dv_table"); + + // Act + SnapshotDeltaLakeTable conversionAction = + DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeKernelTable( + spark, newTableIdentifier, sourceTableLocation); + conversionAction.execute(); + + // Assert + checkLatestSnapshotIntegrity(sourceTable, newTableIdentifier); + checkTagContentAndOrder(sourceTable, sourceTableLocation, newTableIdentifier, 0); + checkIcebergTableLocation(newTableIdentifier, sourceTableLocation); + } + + private void checkLatestSnapshotIntegrity( + String deltaTableIdentifier, String icebergTableIdentifier) { + checkSnapshotIntegrityForQuery( + "SELECT * FROM " + deltaTableIdentifier, "SELECT * FROM " + icebergTableIdentifier); + } + + private void checkSnapshotIntegrityForQuery(String deltaSql, String icebergSql) { + List deltaTableContents = spark.sql(deltaSql).collectAsList(); + List icebergTableContents = spark.sql(icebergSql).collectAsList(); + + assertThat(deltaTableContents).hasSize(icebergTableContents.size()); + assertThat(icebergTableContents).containsExactlyInAnyOrderElementsOf(deltaTableContents); + } + + private void checkTagContentAndOrder( + String deltaTableIdentifier, + String deltaTableLocation, + String icebergTableIdentifier, + long firstConstructableVersion) { + DefaultEngine deltaEngine = DefaultEngine.create(spark.sessionState().newHadoopConf()); + io.delta.kernel.Table deltaTable = + io.delta.kernel.Table.forPath(deltaEngine, deltaTableLocation); + io.delta.kernel.Snapshot latestSnapshot = deltaTable.getLatestSnapshot(deltaEngine); + long currentVersion = latestSnapshot.getVersion(); + Table icebergTable = getIcebergTable(icebergTableIdentifier); + Map icebergSnapshotRefs = icebergTable.refs(); + List icebergSnapshots = Lists.newArrayList(icebergTable.snapshots()); + + assertThat(icebergSnapshots).hasSize((int) (currentVersion - firstConstructableVersion + 1)); + + for (int i = 0; i < icebergSnapshots.size(); i++) { + long deltaVersion = firstConstructableVersion + i; + Snapshot currentIcebergSnapshot = icebergSnapshots.get(i); + + String expectedVersionTag = "delta-version-" + deltaVersion; + icebergSnapshotRefs.get(expectedVersionTag); + assertThat(icebergSnapshotRefs.get(expectedVersionTag)).isNotNull(); + assertThat(icebergSnapshotRefs.get(expectedVersionTag).isTag()).isTrue(); + assertThat(icebergSnapshotRefs.get(expectedVersionTag).snapshotId()) + .isEqualTo(currentIcebergSnapshot.snapshotId()); + + long deltaVersionTs = + deltaTable.getSnapshotAsOfVersion(deltaEngine, deltaVersion).getTimestamp(deltaEngine); + String expectedTimestampTag = "delta-ts-" + deltaVersionTs; + + assertThat(icebergSnapshotRefs.get(expectedTimestampTag)).isNotNull(); + assertThat(icebergSnapshotRefs.get(expectedTimestampTag).isTag()).isTrue(); + assertThat(icebergSnapshotRefs.get(expectedTimestampTag).snapshotId()) + .isEqualTo(currentIcebergSnapshot.snapshotId()); + checkSnapshotIntegrityForQuery( + "SELECT * FROM " + deltaTableIdentifier + " VERSION AS OF " + deltaVersion, + "SELECT * FROM " + + icebergTableIdentifier + + " VERSION AS OF '" + + expectedVersionTag + + "'"); + } + } + + private void writeDeltaTable( + Dataset df, String identifier, String path, String... partitionColumns) { + spark.sql(String.format("DROP TABLE IF EXISTS %s", identifier)); + DataFrameWriter delta = + df.write() + .format("delta") + .mode(SaveMode.Append) + .option("path", path) + .option("delta.enableInCommitTimestamps", "true") + .option("delta.enableRowTracking", "true"); // Increase delta writer version to 7 + + if (partitionColumns.length > 0) { + delta = delta.partitionBy(partitionColumns); + } + delta.saveAsTable(identifier); + } + + private String toFullTableName(String catalogName, String dest) { + if (catalogName.equals(DEFAULT_SPARK_CATALOG)) { + return NAMESPACE + "." + catalogName + "_" + dest; + } + return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest; + } + + private boolean deleteDeltaLogFile(String logName) throws IOException { + String tablePath = sourceLocation.toPath().toString(); + return Files.deleteIfExists(Paths.get(tablePath, "/_delta_log/", logName)); + } + + private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) { + Table icebergTable = getIcebergTable(icebergTableIdentifier); + assertThat(icebergTable.location()) + .isEqualTo(LocationUtil.stripTrailingSlash(expectedLocation)); + } + + 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())); + } +} diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeKernelTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeKernelTableAction.java new file mode 100644 index 000000000000..3cf752e6f37e --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeKernelTableAction.java @@ -0,0 +1,578 @@ +/* + * 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.kernel.Scan; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.data.FilteredColumnarBatch; +import io.delta.kernel.data.Row; +import io.delta.kernel.defaults.engine.DefaultEngine; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.TableNotFoundException; +import io.delta.kernel.internal.DeltaHistoryManager; +import io.delta.kernel.internal.DeltaLogActionUtils; +import io.delta.kernel.internal.SnapshotImpl; +import io.delta.kernel.internal.TableImpl; +import io.delta.kernel.internal.actions.AddFile; +import io.delta.kernel.internal.actions.RemoveFile; +import io.delta.kernel.internal.fs.Path; +import io.delta.kernel.internal.util.VectorUtils; +import io.delta.kernel.types.StructType; +import io.delta.kernel.utils.CloseableIterator; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManageSnapshots; +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.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class BaseSnapshotDeltaLakeKernelTableAction implements SnapshotDeltaLakeTable { + private static final Logger LOG = + LoggerFactory.getLogger(BaseSnapshotDeltaLakeKernelTableAction.class); + + 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 DELTA_VERSION_TAG_PREFIX = "delta-version-"; + private static final String DELTA_TIMESTAMP_TAG_PREFIX = "delta-ts-"; + + private final ImmutableMap.Builder icebergPropertiesBuilder = + ImmutableMap.builder(); + + private final String deltaTableLocation; + private Engine deltaEngine; + private TableImpl deltaTable; + + private Catalog icebergCatalog; + private TableIdentifier newTableIdentifier; + private String newTableLocation; + private HadoopFileIO deltaLakeFileIO; + private DeletionVectorConverter deletionVectorConverter; + private OutputFileFactory icebergFileFactory; + + BaseSnapshotDeltaLakeKernelTableAction(String deltaTableLocation) { + this.deltaTableLocation = deltaTableLocation; + this.newTableLocation = deltaTableLocation; + } + + @Override + public SnapshotDeltaLakeTable tableProperties(Map properties) { + icebergPropertiesBuilder.putAll(properties); + return this; + } + + @Override + public SnapshotDeltaLakeTable tableProperty(String name, String value) { + icebergPropertiesBuilder.put(name, value); + return this; + } + + @Override + public SnapshotDeltaLakeTable tableLocation(String location) { + newTableLocation = location; + return this; + } + + @Override + public SnapshotDeltaLakeTable as(TableIdentifier identifier) { + newTableIdentifier = identifier; + return this; + } + + @Override + public SnapshotDeltaLakeTable icebergCatalog(Catalog catalog) { + icebergCatalog = catalog; + return this; + } + + @Override + public SnapshotDeltaLakeTable deltaLakeConfiguration(Configuration conf) { + deltaEngine = DefaultEngine.create(conf); + deltaLakeFileIO = new HadoopFileIO(conf); + deltaTable = (TableImpl) Table.forPath(deltaEngine, deltaTableLocation); + deletionVectorConverter = new DeletionVectorConverter(deltaEngine, deltaTableLocation); + 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( + deltaTable != null && deltaLakeFileIO != null, + "Make sure to configure the action with a valid deltaLakeConfiguration"); + assertDeltaColumnMappingDisabled( + "Conversion of Delta Lake tables with columnMapping feature is not supported."); + + final long latestDeltaVersion = getLatestDeltaSnapshot().getVersion(); + final long minimalAvailableDeltaVersion = getEarliestRecreatableDeltaLog(); + + SnapshotImpl initialDeltaSnapshot = getDeltaSnapshotAsOfVersion(minimalAvailableDeltaVersion); + + LOG.info( + "Converting Delta Lake table at {} from version {} to version {} into Iceberg table {} ...", + deltaTableLocation, + minimalAvailableDeltaVersion, + latestDeltaVersion, + newTableIdentifier); + + Schema icebergSchema = convertToIcebergSchema(initialDeltaSnapshot.getSchema()); + PartitionSpec partitionSpec = + buildPartitionSpec(icebergSchema, initialDeltaSnapshot.getPartitionColumnNames()); + + Transaction transaction = + icebergCatalog.newCreateTableTransaction( + newTableIdentifier, + icebergSchema, + partitionSpec, + newTableLocation, + buildTablePropertiesWithDelta(initialDeltaSnapshot, deltaTableLocation)); + setDefaultNamingMapping(transaction); + + icebergFileFactory = + OutputFileFactory.builderFor(transaction.table(), 1, 1).format(FileFormat.PUFFIN).build(); + + Set processedDataFiles = Sets.newHashSet(); + try { + commitDeltaSnapshotToIcebergTransaction( + initialDeltaSnapshot, transaction, processedDataFiles); + convertEachDeltaVersion( + minimalAvailableDeltaVersion + 1, latestDeltaVersion, transaction, processedDataFiles); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + transaction.commitTransaction(); + + LOG.info( + "Successfully created Iceberg table {} from Delta Lake table at {}, processed data file count: {}", + newTableIdentifier, + deltaTableLocation, + processedDataFiles.size()); + return processedDataFiles::size; + } + + private void commitDeltaSnapshotToIcebergTransaction( + SnapshotImpl deltaSnapshot, Transaction transaction, Set processedDataFiles) + throws IOException { + Scan scan = deltaSnapshot.getScanBuilder().build(); + try (CloseableIterator changes = scan.getScanFiles(deltaEngine)) { + while (changes.hasNext()) { + FilteredColumnarBatch columnarBatch = changes.next(); + commitDeltaColumnarBatchToIcebergTransaction( + columnarBatch.getData(), transaction, processedDataFiles); + } + tagCurrentSnapshot( + deltaSnapshot.getVersion(), deltaSnapshot.getTimestamp(deltaEngine), transaction); + } + } + + /** + * Current implementation uses the schema conversion mapping based on the logical names. Delta + * Lake supports three column mapping modes: none, name, id. So, the renames with columnMapping + * feature can lead to data corruption. + */ + private void assertDeltaColumnMappingDisabled(String errorMessage) { + Map configuration = getLatestDeltaSnapshot().getMetadata().getConfiguration(); + String columnMappingMode = configuration.getOrDefault("delta.columnMapping.mode", "none"); + if (!"none".equals(columnMappingMode)) { + throw new UnsupportedOperationException(errorMessage); + } + } + + private static void setDefaultNamingMapping(Transaction transaction) { + transaction + .table() + .updateProperties() + .set( + TableProperties.DEFAULT_NAME_MAPPING, + NameMappingParser.toJson(MappingUtil.create(transaction.table().schema()))) + .commit(); + } + + private SnapshotImpl getDeltaSnapshotAsOfVersion(long earliestDeltaFile) { + Snapshot snapshot = deltaTable.getSnapshotAsOfVersion(deltaEngine, earliestDeltaFile); + assertSnapshotImpl(snapshot); + return (SnapshotImpl) snapshot; + } + + private void convertEachDeltaVersion( + long initialDeltaVersion, + long latestDeltaVersion, + Transaction transaction, + Set processedDataFiles) + throws IOException { + + for (long currDeltaVersion = initialDeltaVersion; + currDeltaVersion <= latestDeltaVersion; + currDeltaVersion++) { + try (CloseableIterator changes = + deltaTable.getChanges( + deltaEngine, + currDeltaVersion, + currDeltaVersion, + Set.of(DeltaLogActionUtils.DeltaAction.values()))) { + while (changes.hasNext()) { + ColumnarBatch columnarBatch = changes.next(); + + Long commitTimestamp = + commitDeltaColumnarBatchToIcebergTransaction( + columnarBatch, transaction, processedDataFiles); + tagCurrentSnapshot(currDeltaVersion, commitTimestamp, transaction); + } + } + } + } + + /** + * Convert each delta log {@code ColumnarBatch} to Iceberg action and commit to the given {@code + * Transaction}. The complete spec of delta + * actions.
+ * Supported: + *
  • Add + * + * @return number of added data files + */ + private Long commitDeltaColumnarBatchToIcebergTransaction( + ColumnarBatch columnarBatch, Transaction transaction, Set processedDataFiles) + throws IOException { + + Long originalCommitTimestamp = null; + List dataFilesToAdd = Lists.newArrayList(); + List dataFilesToRemove = Lists.newArrayList(); + List deleteFilesToAdd = Lists.newArrayList(); + + try (CloseableIterator rows = columnarBatch.getRows()) { + while (rows.hasNext()) { + Row row = rows.next(); + if (DeltaLakeActionsTranslationUtil.isCommitInfo(row)) { + Row commitInfo = row.getStruct(row.getSchema().indexOf("commitInfo")); + originalCommitTimestamp = commitInfo.getLong(commitInfo.getSchema().indexOf("timestamp")); + } else if (DeltaLakeActionsTranslationUtil.isAdd(row)) { + AddFile addFile = DeltaLakeActionsTranslationUtil.toAdd(row); + + DataFile dataFile = buildDataFileFromAddDeltaAction(addFile, transaction); + dataFilesToAdd.add(dataFile); + + List deleteFiles = + convertDeltaDVsToIcebergDVs(transaction.table().spec(), addFile, dataFile); + + deleteFilesToAdd.addAll(deleteFiles); + processedDataFiles.add(dataFile.location()); + } else if (DeltaLakeActionsTranslationUtil.isRemove(row)) { + RemoveFile remove = DeltaLakeActionsTranslationUtil.toRemove(row); + + DataFile dataFile = buildDataFileFromRemoveDeltaAction(remove, transaction); + dataFilesToRemove.add(dataFile); + processedDataFiles.add(dataFile.location()); + } + } + } + + // TODO support more actions + commitIcebergTransaction(transaction, dataFilesToAdd, dataFilesToRemove, deleteFilesToAdd); + + return originalCommitTimestamp; + } + + private List convertDeltaDVsToIcebergDVs( + PartitionSpec partitionSpec, AddFile addFile, DataFile dataFile) throws IOException { + if (addFile.getDeletionVector().isEmpty()) { + return List.of(); + } + + DVFileWriter dvWriter = new BaseDVFileWriter(icebergFileFactory, path -> null); + try (DVFileWriter closeableWriter = dvWriter) { + long[] positions = + deletionVectorConverter.readDeltaDVPositions(addFile.getDeletionVector().get()); + for (long deletedRowIndex : positions) { + closeableWriter.delete( + dataFile.location(), deletedRowIndex, partitionSpec, dataFile.partition()); + } + } + + return dvWriter.result().deleteFiles(); + } + + /** + * CASES: + * + *
      + *
    1. Append only + *
    2. Delete only + *
    3. Append and Delete => overwrite + *
    4. RowDelta with deletes. + *
    5. No Append, No Delete => No data changes, append tag or snapshot. + *
    + */ + private static void commitIcebergTransaction( + Transaction transaction, + List dataFilesToAdd, + List dataFilesToRemove, + List deleteFilesToAdd) { + if (!deleteFilesToAdd.isEmpty()) { + // Row Delta + RowDelta rowDelta = transaction.newRowDelta(); + // Avoid validation for multiple DVs added in transaction + // org/apache/iceberg/MergingSnapshotProducer.java:854 + // since we do the conversion sequentially in a single Iceberg transaction + rowDelta.validateFromSnapshot(transaction.table().currentSnapshot().snapshotId()); + + dataFilesToAdd.forEach(rowDelta::addRows); + dataFilesToRemove.forEach(rowDelta::removeRows); + deleteFilesToAdd.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } else if (!dataFilesToAdd.isEmpty() && dataFilesToRemove.isEmpty()) { + // Append only + AppendFiles appendFiles = transaction.newAppend(); + dataFilesToAdd.forEach(appendFiles::appendFile); + appendFiles.commit(); + } else if (dataFilesToAdd.isEmpty() && !dataFilesToRemove.isEmpty()) { + // Delete only + DeleteFiles deleteFiles = transaction.newDelete(); + dataFilesToRemove.forEach(deleteFiles::deleteFile); + deleteFiles.commit(); + } else if (!dataFilesToAdd.isEmpty() && !dataFilesToRemove.isEmpty()) { + // Overwrite + OverwriteFiles overwriteFiles = transaction.newOverwrite(); + dataFilesToAdd.forEach(overwriteFiles::addFile); + dataFilesToRemove.forEach(overwriteFiles::deleteFile); + overwriteFiles.commit(); + } else { + // Tag case + transaction.newAppend().commit(); + } + } + + private DataFile buildDataFileFromAddDeltaAction(AddFile addFile, Transaction transaction) { + String path = addFile.getPath(); + long dataFileSize = addFile.getSize(); + String fullFilePath = getFullFilePath(path, deltaTable.getPath(deltaEngine)); + + InputFile inputDataFile = deltaLakeFileIO.newInputFile(fullFilePath); + if (!inputDataFile.exists()) { + throw new NotFoundException( + "File %s is referenced in the logs of Delta Lake table at %s, but cannot be found in the storage", + fullFilePath, deltaTableLocation); + } + + MetricsConfig metricsConfig = MetricsConfig.forTable(transaction.table()); + String nameMappingString = + transaction.table().properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + Metrics metrics = ParquetUtil.fileMetrics(inputDataFile, metricsConfig, nameMapping); + + Map partitionValues = VectorUtils.toJavaMap(addFile.getPartitionValues()); + PartitionSpec partitionSpec = transaction.table().spec(); + List partitionValueList = + partitionSpec.fields().stream() + .map(PartitionField::name) + .map(partitionValues::get) + .collect(Collectors.toList()); + + return DataFiles.builder(partitionSpec) + .withPath(fullFilePath) + .withFormat(FileFormat.PARQUET) // Delta supports only parquet datafiles + .withFileSizeInBytes(dataFileSize) + .withMetrics(metrics) + .withPartitionValues(partitionValueList) + .build(); + } + + private DataFile buildDataFileFromRemoveDeltaAction( + RemoveFile removeFile, Transaction transaction) { + String path = removeFile.getPath(); + String fullFilePath = getFullFilePath(path, deltaTable.getPath(deltaEngine)); + + InputFile inputDataFile = deltaLakeFileIO.newInputFile(fullFilePath); + if (!inputDataFile.exists()) { + throw new NotFoundException( + "File %s is referenced in the logs of Delta Lake table at %s, but cannot be found in the storage", + fullFilePath, deltaTableLocation); + } + + MetricsConfig metricsConfig = MetricsConfig.forTable(transaction.table()); + String nameMappingString = + transaction.table().properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + Metrics metrics = ParquetUtil.fileMetrics(inputDataFile, metricsConfig, nameMapping); + + Optional> partitionMap = + removeFile.getPartitionValues().map(VectorUtils::toJavaMap); + Map partitionValues = partitionMap.orElseGet(Map::of); + PartitionSpec partitionSpec = transaction.table().spec(); + List partitionValueList = + partitionSpec.fields().stream() + .map(PartitionField::name) + .map(partitionValues::get) + .collect(Collectors.toList()); + + return DataFiles.builder(partitionSpec) + .withPath(fullFilePath) + .withFormat(FileFormat.PARQUET) // Delta supports only parquet datafiles + .withMetrics(metrics) + .withFileSizeInBytes(inputDataFile.getLength()) + .withPartitionValues(partitionValueList) + .build(); + } + + @Nonnull + private static Schema convertToIcebergSchema(StructType deltaSchema) { + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + return new Schema(converted.asNestedType().asStructType().fields()); + } + + private long getEarliestRecreatableDeltaLog() { + try { + // "_delta_log" is unmodifiable logs location + return DeltaHistoryManager.getEarliestRecreatableCommit( + deltaEngine, new Path(deltaTableLocation, "_delta_log")); + } catch (TableNotFoundException e) { + throw deltaTableNotFoundException(e); + } + } + + private SnapshotImpl getLatestDeltaSnapshot() { + Snapshot latestSnapshot; + try { + latestSnapshot = deltaTable.getLatestSnapshot(deltaEngine); + + assertSnapshotImpl(latestSnapshot); + + return (SnapshotImpl) latestSnapshot; + } catch (TableNotFoundException e) { + throw deltaTableNotFoundException(e); + } + } + + private PartitionSpec buildPartitionSpec(Schema schema, List partitionNames) { + if (partitionNames.isEmpty()) { + return PartitionSpec.unpartitioned(); + } + + PartitionSpec.Builder builder = PartitionSpec.builderFor(schema); + for (String partitionName : partitionNames) { + builder.identity(partitionName); + } + return builder.build(); + } + + private Map buildTablePropertiesWithDelta( + SnapshotImpl deltaSnapshot, String originalLocation) { + icebergPropertiesBuilder.put(SNAPSHOT_SOURCE_PROP, DELTA_SOURCE_VALUE); + icebergPropertiesBuilder.put(ORIGINAL_LOCATION_PROP, originalLocation); + // Always construct Iceber v3 Table + icebergPropertiesBuilder.put(TableProperties.FORMAT_VERSION, "3"); + + Map configuration = deltaSnapshot.getMetadata().getConfiguration(); + icebergPropertiesBuilder.putAll(configuration); + + return icebergPropertiesBuilder.build(); + } + + private void tagCurrentSnapshot( + long deltaVersion, Long deltaVersionTimestamp, Transaction transaction) { + if (transaction.table().currentSnapshot() == null) { + return; // Empty table doesn't have Iceberg snapshot yet + } + long currentSnapshotId = transaction.table().currentSnapshot().snapshotId(); + + ManageSnapshots manageSnapshots = transaction.manageSnapshots(); + manageSnapshots.createTag(DELTA_VERSION_TAG_PREFIX + deltaVersion, currentSnapshotId); + + if (deltaVersionTimestamp != null) { + manageSnapshots.createTag( + DELTA_TIMESTAMP_TAG_PREFIX + deltaVersionTimestamp, currentSnapshotId); + } + manageSnapshots.commit(); + } + + private static void assertSnapshotImpl(Snapshot latestSnapshot) { + if (!(latestSnapshot instanceof SnapshotImpl)) { + throw new IllegalStateException( + "Unsupported impl of delta Snapshot: " + latestSnapshot.getClass()); + } + } + + @Nonnull + private IllegalArgumentException deltaTableNotFoundException(TableNotFoundException exception) { + return new IllegalArgumentException( + String.format( + "Delta Lake table does not exist at the given location: %s", deltaTableLocation), + exception); + } + + private static String getFullFilePath(String path, String tableRoot) { + URI dataFileUri = URI.create(path); + String decodedPath = URLDecoder.decode(path, StandardCharsets.UTF_8); + if (dataFileUri.isAbsolute()) { + return decodedPath; + } else { + return tableRoot + File.separator + decodedPath; + } + } +} diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeletionVectorConverter.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeletionVectorConverter.java new file mode 100644 index 000000000000..7dfe60488e5b --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeletionVectorConverter.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 io.delta.kernel.engine.Engine; +import io.delta.kernel.internal.actions.DeletionVectorDescriptor; +import io.delta.kernel.internal.deletionvectors.DeletionVectorUtils; +import io.delta.kernel.internal.deletionvectors.RoaringBitmapArray; +import io.delta.kernel.internal.util.Tuple2; + +public class DeletionVectorConverter { + private final Engine engine; + private final String tablePath; + + public DeletionVectorConverter(Engine engine, String tablePath) { + this.engine = engine; + this.tablePath = tablePath; + } + + public long[] readDeltaDVPositions(DeletionVectorDescriptor descriptor) { + Tuple2 tuple = + DeletionVectorUtils.loadNewDvAndBitmap(engine, tablePath, descriptor); + + return tuple._2.toArray(); + } +} diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeActionsTranslationUtil.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeActionsTranslationUtil.java new file mode 100644 index 000000000000..6a6901fb1c20 --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeActionsTranslationUtil.java @@ -0,0 +1,76 @@ +/* + * 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.kernel.data.Row; +import io.delta.kernel.internal.actions.AddFile; +import io.delta.kernel.internal.actions.RemoveFile; + +/** + * Util class helps to handle json operations for delta action + */ +class DeltaLakeActionsTranslationUtil { + private DeltaLakeActionsTranslationUtil() {} + + public static boolean isAdd(Row row) { + return isNotNullAt(row, "add"); + } + + public static AddFile toAdd(Row row) { + return new AddFile(row.getStruct(row.getSchema().indexOf("add"))); + } + + public static boolean isRemove(Row row) { + return isNotNullAt(row, "remove"); + } + + public static RemoveFile toRemove(Row row) { + return new RemoveFile(row.getStruct(row.getSchema().indexOf("remove"))); + } + + public static boolean isMetaData(Row row) { + return isNotNullAt(row, "metaData"); + } + + public static boolean isTxn(Row row) { + return isNotNullAt(row, "txn"); + } + + public static boolean isProtocol(Row row) { + return isNotNullAt(row, "protocol"); + } + + public static boolean isCdc(Row row) { + return isNotNullAt(row, "cdc"); + } + + public static boolean isCommitInfo(Row row) { + return isNotNullAt(row, "commitInfo"); + } + + private static boolean isNotNullAt(Row row, String fieldName) { + int position = getOrdinal(row, fieldName); + return position >= 0 && !row.isNullAt(position); + } + + private static int getOrdinal(Row row, String filedName) { + return row.getSchema().indexOf(filedName); + } +} diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeKernelTypeToType.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeKernelTypeToType.java new file mode 100644 index 000000000000..a762b4b092d1 --- /dev/null +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeKernelTypeToType.java @@ -0,0 +1,183 @@ +/* + * 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.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DataType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FieldMetadata; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampNTZType; +import io.delta.kernel.types.TimestampType; +import io.delta.kernel.types.VariantType; +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; + +class DeltaLakeKernelTypeToType { + private final StructType root; + private int nextId = 0; + + DeltaLakeKernelTypeToType(StructType root) { + this.root = root; + this.nextId = root.fields().size(); + } + + public Type convertType() { + return convertType(root); + } + + private Type convertType(DataType type) { + if (type instanceof StructType) { + List fields = ((StructType) type).fields(); + List fieldResults = Lists.newArrayListWithExpectedSize(fields.size()); + + for (StructField field : fields) { + fieldResults.add(convertType(field.getDataType())); + } + + return struct((StructType) type, fieldResults); + + } else if (type instanceof MapType) { + return map( + (MapType) type, + convertType(((MapType) type).getKeyType()), + convertType(((MapType) type).getValueType())); + + } else if (type instanceof ArrayType) { + return array((ArrayType) type, convertType(((ArrayType) type).getElementType())); + + } else { + return atomic(type); + } + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @SuppressWarnings("ReferenceEquality") + private Type struct(StructType struct, List types) { + List fields = struct.fields(); + List newFields = Lists.newArrayListWithExpectedSize(fields.size()); + boolean isRoot = root == struct; + + for (int i = 0; i < fields.size(); i += 1) { + StructField field = fields.get(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(); + } + + // Delta Kernel StructField metadata is a Map + FieldMetadata metadata = field.getMetadata(); + String doc = metadata.contains("comment") ? metadata.getString("comment") : 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); + } + + private Type array(ArrayType array, Type elementType) { + if (array.containsNull()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + private Type map(MapType map, Type keyType, Type valueType) { + if (map.isValueContainsNull()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private Type atomic(DataType atomic) { + 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 TimestampNTZType) { + return Types.TimestampType.withoutZone(); + + } else if (atomic instanceof DecimalType) { + return Types.DecimalType.of( + ((DecimalType) atomic).getPrecision(), ((DecimalType) atomic).getScale()); + + } else if (atomic instanceof BinaryType) { + return Types.BinaryType.get(); + + } else if (atomic instanceof VariantType) { + return Types.VariantType.get(); + } + + throw new ValidationException("Not a supported type: %s", atomic.toString()); + } +} diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeKernelTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeKernelTableAction.java new file mode 100644 index 000000000000..96c61248db19 --- /dev/null +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeKernelTableAction.java @@ -0,0 +1,383 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import io.delta.kernel.Operation; +import io.delta.kernel.Table; +import io.delta.kernel.Transaction; +import io.delta.kernel.TransactionBuilder; +import io.delta.kernel.defaults.engine.DefaultEngine; +import io.delta.kernel.engine.Engine; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructType; +import io.delta.kernel.utils.CloseableIterable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.StandardCopyOption; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestBaseSnapshotDeltaLakeKernelTableAction { + private static final String DELTA_GOLDEN_TABLES_ROOT = "delta/golden/"; + + @TempDir private File sourceDeltaFolder; + @TempDir private File icebergWarehouseFolder; + private String sourceTableLocation; + private final Configuration testHadoopConf = new Configuration(); + private String newTableLocation; + private Catalog testCatalog; + + @BeforeEach + public void before() throws IOException { + sourceTableLocation = sourceDeltaFolder.toURI().toString(); + + Path icebergTablePath = icebergWarehouseFolder.toPath().resolve("iceberg_table"); + newTableLocation = icebergTablePath.toString(); + Files.createDirectories(icebergTablePath); + + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, icebergWarehouseFolder.toString()); + + HadoopCatalog icebergCatalog = new HadoopCatalog(); + icebergCatalog.setConf(new Configuration()); + icebergCatalog.initialize("hadoop_catalog", properties); + testCatalog = icebergCatalog; + } + + @Test + public void testRequiredTableIdentifier() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .icebergCatalog(testCatalog) + .deltaLakeConfiguration(testHadoopConf) + .tableLocation(newTableLocation); + 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 + public void testRequiredIcebergCatalog() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .deltaLakeConfiguration(testHadoopConf) + .tableLocation(newTableLocation); + 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 + public void testRequiredDeltaLakeConfiguration() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Make sure to configure the action with a valid deltaLakeConfiguration"); + } + + @Test + public void testDeltaTableColumnMappingFeatureDisabled() throws Exception { + Engine engine = DefaultEngine.create(testHadoopConf); + Transaction txn = + createEmptyDeltaTableTransaction(engine) + .withTableProperties(engine, Map.of("delta.columnMapping.mode", "name")) + .build(engine); + txn.commit(engine, CloseableIterable.emptyIterable()); + + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(TableIdentifier.of("iceberg_table")) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + // Act & check + assertThatThrownBy(testAction::execute) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Conversion of Delta Lake tables with columnMapping feature is not supported."); + } + + @Test + public void testEmptyTableConversion() { + Engine engine = DefaultEngine.create(testHadoopConf); + createEmptyDeltaTableTransaction(engine) + .build(engine) + .commit(engine, CloseableIterable.emptyIterable()); + + TableIdentifier icebergTable = TableIdentifier.of("iceberg_table"); + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(icebergTable) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + assertThat(testCatalog.tableExists(icebergTable)).isFalse(); + + testAction.execute(); + + assertThat(testCatalog.tableExists(icebergTable)).isTrue(); + + org.apache.iceberg.Table table = testCatalog.loadTable(icebergTable); + assertThat(((BaseTable) table).operations().current().formatVersion()).isEqualTo(3); + } + + @Test + public void testTableCreated() throws Exception { + loadDeltaLakeGoldenTable("basic-decimal-table"); + + TableIdentifier icebergTable = TableIdentifier.of("iceberg_table"); + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(icebergTable) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + assertThat(testCatalog.tableExists(icebergTable)).isFalse(); + + testAction.execute(); + + assertThat(testCatalog.tableExists(icebergTable)).isTrue(); + } + + @Test + public void testDefaultTableProperties() throws Exception { + loadDeltaLakeGoldenTable("basic-decimal-table"); + + TableIdentifier icebergTableIdentifier = TableIdentifier.of("iceberg_table"); + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(icebergTableIdentifier) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + assertThat(testCatalog.tableExists(icebergTableIdentifier)).isFalse(); + + testAction.execute(); + + assertThat(testCatalog.tableExists(icebergTableIdentifier)).isTrue(); + + Map properties = testCatalog.loadTable(icebergTableIdentifier).properties(); + assertThat(properties.get("original_location")).isEqualTo(sourceTableLocation); + assertThat(properties.get("snapshot_source")).isEqualTo("delta"); + assertThat(properties.containsKey(TableProperties.DEFAULT_NAME_MAPPING)).isTrue(); + } + + @Test + public void testCustomTableProperties() throws Exception { + loadDeltaLakeGoldenTable("basic-decimal-table"); + + TableIdentifier icebergTableIdentifier = TableIdentifier.of("iceberg_table"); + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(icebergTableIdentifier) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation) + .tableProperty("custom_prop_1", "custom val 1") + .tableProperty("custom_prop_2", "custom val 2") + .tableProperties( + ImmutableMap.of( + "custom_map_prop1", "val 1", + "custom_map_prop2", "val 2")); + + assertThat(testCatalog.tableExists(icebergTableIdentifier)).isFalse(); + + testAction.execute(); + + assertThat(testCatalog.tableExists(icebergTableIdentifier)).isTrue(); + + Map properties = testCatalog.loadTable(icebergTableIdentifier).properties(); + assertThat(properties.get("custom_prop_1")).isEqualTo("custom val 1"); + assertThat(properties.get("custom_prop_2")).isEqualTo("custom val 2"); + assertThat(properties.get("custom_map_prop1")).isEqualTo("val 1"); + assertThat(properties.get("custom_map_prop2")).isEqualTo("val 2"); + } + + @Test + void testDeltaTableNotExist() { + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(TableIdentifier.of("test", "test")) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + assertThatThrownBy(testAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Delta Lake table does not exist at the given location: %s", sourceTableLocation); + } + + @Test + public void testDeltaTableDVSupported() throws Exception { + loadDeltaLakeGoldenTable("dv-partitioned-with-checkpoint"); + Engine engine = DefaultEngine.create(testHadoopConf); + Table deltaTable = Table.forPath(engine, sourceTableLocation); + + io.delta.kernel.internal.SnapshotImpl deltaSnapshot = + (io.delta.kernel.internal.SnapshotImpl) deltaTable.getLatestSnapshot(engine); + assertThat( + deltaSnapshot + .getMetadata() + .getConfiguration() + .getOrDefault("delta.enableDeletionVectors", "false")) + .isEqualTo("true"); + + TableIdentifier icebergTableIdentifier = TableIdentifier.of("iceberg_table"); + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(icebergTableIdentifier) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + // Act + testAction.execute(); + + org.apache.iceberg.Table icebergTable = testCatalog.loadTable(icebergTableIdentifier); + try (org.apache.iceberg.io.CloseableIterable tasks = + icebergTable.newScan().planFiles()) { + boolean hasDeleteFiles = false; + for (org.apache.iceberg.FileScanTask task : tasks) { + for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { + assertThat(ContentFileUtil.isDV(deleteFile)).isTrue(); + hasDeleteFiles = true; + } + } + assertThat(hasDeleteFiles).isTrue(); + } + } + + static List deltaGoldenTableNames() throws Exception { + Path rootPath = + Paths.get( + TestBaseSnapshotDeltaLakeKernelTableAction.class + .getClassLoader() + .getResource(DELTA_GOLDEN_TABLES_ROOT) + .toURI()); + + try (Stream stream = Files.list(rootPath)) { + return stream + .filter(Files::isDirectory) + .map(path -> path.getFileName().toString()) + .collect(Collectors.toList()); + } + } + + @ParameterizedTest + @MethodSource("deltaGoldenTableNames") + void goldenDeltaTableConversion(String deltaTableName) throws Exception { + loadDeltaLakeGoldenTable(deltaTableName); + + SnapshotDeltaLakeTable testAction = + new BaseSnapshotDeltaLakeKernelTableAction(sourceTableLocation) + .as(TableIdentifier.of("iceberg_table")) + .deltaLakeConfiguration(testHadoopConf) + .icebergCatalog(testCatalog) + .tableLocation(newTableLocation); + + testAction.execute(); + } + + private TransactionBuilder createEmptyDeltaTableTransaction(Engine engine) { + StructType schema = + new StructType().add("id", IntegerType.INTEGER).add("status", StringType.STRING); + Table table = Table.forPath(engine, sourceTableLocation); + + return table + .createTransactionBuilder(engine, "Delta-Iceberg-Converter-Tests", Operation.CREATE_TABLE) + .withSchema(engine, schema); + } + + private void loadDeltaLakeGoldenTable(String goldenTableName) throws Exception { + String goldenTableFolderPath = DELTA_GOLDEN_TABLES_ROOT + goldenTableName; + URI resourceUri = + Objects.requireNonNull( + getClass().getClassLoader().getResource(goldenTableFolderPath), + "Could not find Delta table test table path: " + goldenTableFolderPath) + .toURI(); + + Path source = Paths.get(resourceUri); + Path target = sourceDeltaFolder.toPath(); + copyDirectory(source, target); + } + + private void copyDirectory(Path source, Path target) throws IOException { + Files.walkFileTree( + source, + new SimpleFileVisitor<>() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) + throws IOException { + Path targetDir = target.resolve(source.relativize(dir)); + if (!Files.exists(targetDir)) { + Files.createDirectory(targetDir); + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) + throws IOException { + Files.copy( + file, target.resolve(source.relativize(file)), StandardCopyOption.REPLACE_EXISTING); + return FileVisitResult.CONTINUE; + } + }); + } +} diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeKernelTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeKernelTypeToType.java new file mode 100644 index 000000000000..c9012c52e032 --- /dev/null +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeKernelTypeToType.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.delta; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.delta.kernel.types.ArrayType; +import io.delta.kernel.types.BinaryType; +import io.delta.kernel.types.BooleanType; +import io.delta.kernel.types.ByteType; +import io.delta.kernel.types.DateType; +import io.delta.kernel.types.DecimalType; +import io.delta.kernel.types.DoubleType; +import io.delta.kernel.types.FloatType; +import io.delta.kernel.types.IntegerType; +import io.delta.kernel.types.LongType; +import io.delta.kernel.types.MapType; +import io.delta.kernel.types.ShortType; +import io.delta.kernel.types.StringType; +import io.delta.kernel.types.StructField; +import io.delta.kernel.types.StructType; +import io.delta.kernel.types.TimestampNTZType; +import io.delta.kernel.types.TimestampType; +import io.delta.kernel.types.VariantType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestDeltaLakeKernelTypeToType { + private StructType deltaAtomicSchema; + + @BeforeEach + public void constructDeltaLakeSchema() { + deltaAtomicSchema = + new StructType() + .add("testBoolType", BooleanType.BOOLEAN, false) + .add("testIntType", IntegerType.INTEGER) + .add("testLongType", LongType.LONG, false) + .add("testFloatType", FloatType.FLOAT) + .add("testDoubleType", DoubleType.DOUBLE, false) + .add("testStringType", StringType.STRING) + .add("testBinaryType", BinaryType.BINARY, false) + .add("testDateType", DateType.DATE) + .add("testDecimalType", DecimalType.USER_DEFAULT, false) + .add("testDecimalMaxType", new DecimalType(38, 38)); + } + + @Test + public void testAtomicTypeConversion() { + Type converted = new DeltaLakeKernelTypeToType(deltaAtomicSchema).convertType(); + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertToAtomicSchema(convertedSchema); + } + + private void assertToAtomicSchema(Schema convertedSchema) { + assertThat(convertedSchema.columns().size()).isEqualTo(deltaAtomicSchema.fields().size()); + for (StructField field : deltaAtomicSchema.fields()) { + assertThat(convertedSchema.findField(field.getName())).isNotNull(); + assertThat(convertedSchema.findField(field.getName()).isOptional()) + .isEqualTo(field.isNullable()); + } + + assertThat(convertedSchema.findType("testBoolType")).isInstanceOf(Types.BooleanType.class); + assertThat(convertedSchema.findType("testIntType")).isInstanceOf(Types.IntegerType.class); + assertThat(convertedSchema.findType("testLongType")).isInstanceOf(Types.LongType.class); + assertThat(convertedSchema.findType("testFloatType")).isInstanceOf(Types.FloatType.class); + assertThat(convertedSchema.findType("testDoubleType")).isInstanceOf(Types.DoubleType.class); + assertThat(convertedSchema.findType("testStringType")).isInstanceOf(Types.StringType.class); + assertThat(convertedSchema.findType("testBinaryType")).isInstanceOf(Types.BinaryType.class); + assertThat(convertedSchema.findType("testDateType")).isInstanceOf(Types.DateType.class); + assertThat(convertedSchema.findType("testDecimalType")).isInstanceOf(Types.DecimalType.class); + + assertThat(convertedSchema.findType("testDecimalMaxType")) + .isInstanceOf(Types.DecimalType.class); + Types.DecimalType testDecimalMaxType = + (Types.DecimalType) convertedSchema.findType("testDecimalMaxType"); + assertThat(testDecimalMaxType.precision()).isEqualTo(38); + assertThat(testDecimalMaxType.scale()).isEqualTo(38); + } + + @Test + public void testUpcastConversion() { + StructType deltaSchema = + new StructType() + .add("nullableByte", ByteType.BYTE) + .add("requiredShort", ShortType.SHORT, false); + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("nullableByte")).isInstanceOf(Types.IntegerType.class); + assertThat(convertedSchema.findField("nullableByte").isOptional()).isTrue(); + assertThat(convertedSchema.findType("requiredShort")).isInstanceOf(Types.IntegerType.class); + assertThat(convertedSchema.findField("requiredShort").isRequired()).isTrue(); + } + + @Test + public void testTimestampsConversion() { + StructType deltaSchema = + new StructType() + .add("nullableTimestamp", TimestampType.TIMESTAMP) + .add("requiredTimestampNtz", TimestampNTZType.TIMESTAMP_NTZ, false); + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("nullableTimestamp")) + .isInstanceOf(Types.TimestampType.class); + assertThat( + ((Types.TimestampType) convertedSchema.findType("nullableTimestamp")) + .shouldAdjustToUTC()) + .isTrue(); + assertThat(convertedSchema.findField("nullableTimestamp").isOptional()).isTrue(); + assertThat(convertedSchema.findType("requiredTimestampNtz")) + .isInstanceOf(Types.TimestampType.class); + assertThat( + ((Types.TimestampType) convertedSchema.findType("requiredTimestampNtz")) + .shouldAdjustToUTC()) + .isFalse(); + assertThat(convertedSchema.findField("requiredTimestampNtz").isRequired()).isTrue(); + } + + @Test + public void testVariantConversion() { + StructType deltaSchema = + new StructType() // Spec v3 is required + .add("nullableVariant", VariantType.VARIANT); + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("nullableVariant")).isInstanceOf(Types.VariantType.class); + assertThat(convertedSchema.findField("nullableVariant").isOptional()).isTrue(); + } + + @Test + public void testArrayConversion() { + StructType deltaSchema = + new StructType().add("testDoubleArray", new ArrayType(DoubleType.DOUBLE, true), false); + + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("testDoubleArray")).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField("testDoubleArray").isRequired()).isTrue(); + assertThat(convertedSchema.findType("testDoubleArray").asListType().elementType()) + .isInstanceOf(Types.DoubleType.class); + assertThat(convertedSchema.findType("testDoubleArray").asListType().isElementOptional()) + .isTrue(); + } + + @Test + public void testMapConversion() { + StructType deltaSchema = + new StructType() + .add("testStringLongMap", new MapType(StringType.STRING, LongType.LONG, false), false); + Type converted = new DeltaLakeKernelTypeToType(deltaSchema).convertType(); + + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("testStringLongMap")).isInstanceOf(Types.MapType.class); + assertThat(convertedSchema.findField("testStringLongMap").isRequired()).isTrue(); + assertThat(convertedSchema.findType("testStringLongMap").asMapType().keyType()) + .isInstanceOf(Types.StringType.class); + assertThat(convertedSchema.findType("testStringLongMap").asMapType().valueType()) + .isInstanceOf(Types.LongType.class); + assertThat(convertedSchema.findType("testStringLongMap").asMapType().isValueRequired()) + .isTrue(); + } + + @Test + public void testNestedTypeConversion() { + StructType deltaNestedSchema = + new StructType() + .add("testInnerAtomicSchema", deltaAtomicSchema) + .add("testStructArrayType", new ArrayType(deltaAtomicSchema, true), false) + .add( + "testStringStructMap", + new MapType(StringType.STRING, deltaAtomicSchema, false), + false); + Type converted = new DeltaLakeKernelTypeToType(deltaNestedSchema).convertType(); + Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); + + assertThat(convertedSchema.findType("testInnerAtomicSchema")) + .isInstanceOf(Types.StructType.class); + assertThat(convertedSchema.findField("testInnerAtomicSchema").isOptional()).isTrue(); + assertToAtomicSchema( + convertedSchema.findType("testInnerAtomicSchema").asStructType().asSchema()); + + assertThat(convertedSchema.findType("testStructArrayType")).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField("testStructArrayType").isRequired()).isTrue(); + assertThat(convertedSchema.findType("testStructArrayType").asListType().isElementOptional()) + .isTrue(); + assertThat(convertedSchema.findType("testStructArrayType").asListType().elementType()) + .isInstanceOf(Types.StructType.class); + assertToAtomicSchema( + convertedSchema + .findType("testStructArrayType") + .asListType() + .elementType() + .asStructType() + .asSchema()); + + assertThat(convertedSchema.findType("testStringStructMap")).isInstanceOf(Types.MapType.class); + assertThat(convertedSchema.findField("testStringStructMap").isRequired()).isTrue(); + assertThat(convertedSchema.findType("testStringStructMap").asMapType().keyType()) + .isInstanceOf(Types.StringType.class); + assertThat(convertedSchema.findType("testStringStructMap").asMapType().valueType()) + .isInstanceOf(Types.StructType.class); + assertThat(convertedSchema.findType("testStringStructMap").asMapType().isValueRequired()) + .isTrue(); + assertToAtomicSchema( + convertedSchema + .findType("testStringStructMap") + .asMapType() + .valueType() + .asStructType() + .asSchema()); + } +} diff --git a/delta-lake/src/test/resources/delta/golden/README.md b/delta-lake/src/test/resources/delta/golden/README.md new file mode 100644 index 000000000000..5f39bb15f04c --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/README.md @@ -0,0 +1,69 @@ + + +# Delta Lake Golden Tables + +## Overview +This directory contains the **Delta Lake Golden Tables**, a set of reference data tables used to verify compliance with the Delta Lake protocol. + +These tables cover various features of the Delta Lake specification (e.g., different protocol versions, partition strategies, checkpoints, and data types). They act as a "compliance suite" to ensure that this project's reader/writer implementation can correctly handle data produced by the official Delta Lake reference implementation. + +## Origin & Attribution +These resources were sourced directly from the open-source Delta Lake project. + +* **Source Repository:** [delta-io/delta](https://github.com/delta-io/delta) +* **Module Path:** `connectors/golden-tables/src` +* **License:** Apache License 2.0 (See [LICENSE](https://github.com/delta-io/delta/blob/master/LICENSE.txt) in the project) + +## Purpose +The primary goal of these tables is **integration testing**. + +Each folder represents a specific "edge case" or feature configuration in Delta Lake. By running the conversion against +these tables, we verify that: +1. **Protocol Versioning:** We respect `minReaderVersion` and `minWriterVersion`. +2. **Feature Support:** We can read tables with `checkpoints`, `vacuumed` files, `deletion vectors`, or `column mapping`. +3. **Data Consistency:** We read the correct number of rows and column values as defined by the transaction log (`_delta_log`). + +## Table Inventory & Explanations + +The tables are typically named after the specific feature they test. Below are explanations of common patterns found in this suite: + +### 1. Basic Protocol Tests +* **`primitives` / `all-types`**: Tables containing every supported primitive data type (INT, LONG, STRING, TIMESTAMP, etc.) to ensure type mapping is correct. + +### 2. Transaction Log Features +* **`checkpoint`**: Tables that have been checkpointed (compaction of the JSON log into Parquet). Tests if the reader can reconstruct state from a checkpoint file + subsequent JSON deltas. +* **`vacuum`**: Tables where old data files have been physically deleted. Tests if the reader correctly ignores files no longer referenced in the log. +* **`streaming`**: Tables created via streaming sinks, often containing specific metadata relating to streaming query progress. + +### 3. Partitioning +* **`partitioned`**: Tables partitioned by one or more columns (e.g., `date`, `region`). Tests if the reader correctly prunes partitions and reconstructs the data. +* **`partitioned-with-null`**: Partitioned tables that include `null` values in the partition keys (a common edge case for directory structures). + +### 4. Advanced Features +* **`delete` / `merge` / `update`**: Tables where data has been modified. Tests if the reader correctly applies the latest version of the data (resolving tombstones/deletions). +* **`change-data-feed`**: Tables with Change Data Feed (CDF) enabled. +* **`deletion-vectors`**: Tables using Deletion Vectors (a newer protocol feature for efficient deletes). *Note: Ensure our reader supports Protocol V3 if using these.* +* **`column-mapping`**: Tables using ID-based or Name-based column mapping (allows renaming columns without rewriting data files). + +## Updating +To update these tables, copy the latest resources from the `master` branch of the [delta-io/delta](https://github.com/delta-io/delta) repository under `connectors/golden-tables/src/test/resources/golden`. + +--- +*Note: This content is for testing purposes only and is derived from the Delta Lake open-source project.* \ No newline at end of file diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/.00000000000000000000.json.crc b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 000000000000..9e29d16b1517 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/.00000000000000000000.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/00000000000000000000.json b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..0aa97e4f4134 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1690853005164,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"part\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputRows":"4","numOutputBytes":"4131"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"451ba03f-e80c-4fda-9bba-8fdfda856925"}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"part\",\"type\":\"decimal(12,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col1\",\"type\":\"decimal(5,2)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col2\",\"type\":\"decimal(10,5)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col3\",\"type\":\"decimal(20,10)\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["part"],"configuration":{},"createdTime":1690852998865}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"add":{"path":"part=-2342342.23423/part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet","partitionValues":{"part":"-2342342.23423"},"size":1032,"modificationTime":1690853004000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"col1\":-999.99,\"col2\":-99999.99999,\"col3\":-9999999999.9999999999},\"maxValues\":{\"col1\":-999.99,\"col2\":-99999.99999,\"col3\":-9999999999.9999999999},\"nullCount\":{\"col1\":0,\"col2\":0,\"col3\":0}}"}} +{"add":{"path":"part=0.00004/part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet","partitionValues":{"part":"0.00004"},"size":1033,"modificationTime":1690853004000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"col1\":0.00,\"col2\":0.00000,\"col3\":0E-10},\"maxValues\":{\"col1\":0.00,\"col2\":0.00000,\"col3\":0E-10},\"nullCount\":{\"col1\":0,\"col2\":0,\"col3\":0}}"}} +{"add":{"path":"part=234.00000/part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet","partitionValues":{"part":"234.00000"},"size":1033,"modificationTime":1690853004000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"col1\":1.00,\"col2\":2.00000,\"col3\":3.0000000000},\"maxValues\":{\"col1\":1.00,\"col2\":2.00000,\"col3\":3.0000000000},\"nullCount\":{\"col1\":0,\"col2\":0,\"col3\":0}}"}} +{"add":{"path":"part=2342222.23454/part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet","partitionValues":{"part":"2342222.23454"},"size":1033,"modificationTime":1690853004000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"col1\":111.11,\"col2\":22222.22222,\"col3\":3333333333.3333333333},\"maxValues\":{\"col1\":111.11,\"col2\":22222.22222,\"col3\":3333333333.3333333333},\"nullCount\":{\"col1\":0,\"col2\":0,\"col3\":0}}"}} diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/.part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/.part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet.crc new file mode 100644 index 000000000000..aef64b9388e9 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/.part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet new file mode 100644 index 000000000000..63820edc4f98 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=-2342342.23423/part-00000-8f850371-9b03-42c4-9d22-f83bc81c9b68.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/.part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/.part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet.crc new file mode 100644 index 000000000000..5646c512042d Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/.part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet new file mode 100644 index 000000000000..18db5c238323 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=0.00004/part-00000-1cb60e36-6cd4-4191-a318-ae9355f877c3.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/.part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/.part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet.crc new file mode 100644 index 000000000000..6c1db4450a80 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/.part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet new file mode 100644 index 000000000000..6645238dbfd9 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=234.00000/part-00000-ac109189-97e5-49af-947f-335a5e46ee5c.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/.part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/.part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet.crc new file mode 100644 index 000000000000..808f4b946f9a Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/.part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet new file mode 100644 index 000000000000..e9095a569c75 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/basic-decimal-table/part=2342222.23454/part-00000-d5a0c70f-7cd3-4d32-a9c0-7171a06547c6.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin.crc new file mode 100644 index 000000000000..63262420a4d7 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin.crc new file mode 100644 index 000000000000..46f91ca2de40 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin.crc new file mode 100644 index 000000000000..63262420a4d7 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin.crc new file mode 100644 index 000000000000..63262420a4d7 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin.crc new file mode 100644 index 000000000000..46f91ca2de40 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin.crc new file mode 100644 index 000000000000..46f91ca2de40 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin.crc new file mode 100644 index 000000000000..46f91ca2de40 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin.crc new file mode 100644 index 000000000000..46f91ca2de40 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin.crc new file mode 100644 index 000000000000..cd273888c4da Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/.deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000000.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 000000000000..9ffa91cf37e5 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000000.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000001.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 000000000000..39b123d83367 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000001.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000002.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 000000000000..f9176d5ecdfc Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000002.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000003.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000003.json.crc new file mode 100644 index 000000000000..104ffec10110 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000003.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000004.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000004.json.crc new file mode 100644 index 000000000000..3021f4cef893 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000004.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000005.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000005.json.crc new file mode 100644 index 000000000000..e77cd3ba978f Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000005.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000006.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000006.json.crc new file mode 100644 index 000000000000..9368bb12e16c Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000006.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000007.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000007.json.crc new file mode 100644 index 000000000000..5d80f0d4a5cb Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000007.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000008.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000008.json.crc new file mode 100644 index 000000000000..cdcd019337c7 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000008.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000009.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000009.json.crc new file mode 100644 index 000000000000..1c4ac3202721 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000009.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc new file mode 100644 index 000000000000..497e0ef59d6e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.checkpoint.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.json.crc new file mode 100644 index 000000000000..68d2344adc54 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000010.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000011.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000011.json.crc new file mode 100644 index 000000000000..e5d00846b73e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000011.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000012.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000012.json.crc new file mode 100644 index 000000000000..31a10434885c Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000012.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000013.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000013.json.crc new file mode 100644 index 000000000000..b05100a34909 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000013.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000014.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000014.json.crc new file mode 100644 index 000000000000..004a177dc855 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000014.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000015.json.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000015.json.crc new file mode 100644 index 000000000000..2ce6c2c16842 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/.00000000000000000015.json.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/._last_checkpoint.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/._last_checkpoint.crc new file mode 100644 index 000000000000..d8cd4225d00e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/._last_checkpoint.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000000.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..056e0b9d3463 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000000.json @@ -0,0 +1,23 @@ +{"commitInfo":{"timestamp":1688691721441,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"part\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"20","numOutputRows":"50","numOutputBytes":"14678"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"bc4bfee2-8faf-4995-bfa1-a3c4930cbf22"}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"part\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["part"],"configuration":{"delta.enableDeletionVectors":"true"},"createdTime":1688691715853}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["deletionVectors"],"writerFeatures":["deletionVectors"]}} +{"add":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","partitionValues":{"part":"0"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":0,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":20,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=1/part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet","partitionValues":{"part":"1"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":1,\"col2\":\"foo1\"},\"maxValues\":{\"col1\":21,\"col2\":\"foo1\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet","partitionValues":{"part":"2"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":2,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":22,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=3/part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet","partitionValues":{"part":"3"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":3,\"col2\":\"foo3\"},\"maxValues\":{\"col1\":23,\"col2\":\"foo3\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=4/part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet","partitionValues":{"part":"4"},"size":735,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":4,\"col2\":\"foo4\"},\"maxValues\":{\"col1\":24,\"col2\":\"foo4\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=5/part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet","partitionValues":{"part":"5"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":5,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":15,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=6/part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet","partitionValues":{"part":"6"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":6,\"col2\":\"foo1\"},\"maxValues\":{\"col1\":16,\"col2\":\"foo1\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=7/part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet","partitionValues":{"part":"7"},"size":732,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":7,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":17,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=8/part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet","partitionValues":{"part":"8"},"size":732,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":8,\"col2\":\"foo3\"},\"maxValues\":{\"col1\":18,\"col2\":\"foo3\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=9/part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet","partitionValues":{"part":"9"},"size":732,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":9,\"col2\":\"foo4\"},\"maxValues\":{\"col1\":19,\"col2\":\"foo4\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=0/part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet","partitionValues":{"part":"0"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":30,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":40,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=1/part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet","partitionValues":{"part":"1"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":31,\"col2\":\"foo1\"},\"maxValues\":{\"col1\":41,\"col2\":\"foo1\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=2/part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet","partitionValues":{"part":"2"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":32,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":42,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=3/part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet","partitionValues":{"part":"3"},"size":731,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":33,\"col2\":\"foo3\"},\"maxValues\":{\"col1\":43,\"col2\":\"foo3\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=4/part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet","partitionValues":{"part":"4"},"size":732,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"col1\":34,\"col2\":\"foo4\"},\"maxValues\":{\"col1\":44,\"col2\":\"foo4\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=5/part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet","partitionValues":{"part":"5"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":25,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":45,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet","partitionValues":{"part":"6"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":26,\"col2\":\"foo1\"},\"maxValues\":{\"col1\":46,\"col2\":\"foo1\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=7/part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet","partitionValues":{"part":"7"},"size":736,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":27,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":47,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet","partitionValues":{"part":"8"},"size":736,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":28,\"col2\":\"foo3\"},\"maxValues\":{\"col1\":48,\"col2\":\"foo3\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} +{"add":{"path":"part=9/part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet","partitionValues":{"part":"9"},"size":736,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":29,\"col2\":\"foo4\"},\"maxValues\":{\"col1\":49,\"col2\":\"foo4\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":true}"}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000001.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..7b4eda2d6843 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691744588,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#463 = 0)\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"13565","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"33e9e33f-49f5-421c-bf62-c3e71c55cd32"}} +{"add":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","partitionValues":{"part":"0"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":0,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":20,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"0X9F0q2<2yJ-f)Gm2!e0","offset":1,"sizeInBytes":34,"cardinality":1}}} +{"remove":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","deletionTimestamp":1688691741121,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"0"},"size":736}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000002.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000002.json new file mode 100644 index 000000000000..685f921fe89b --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000002.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691752166,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#1939 = 2)\"]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"5256","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"a9487f5e-84c9-44d8-a6d6-f7444e8405a9"}} +{"add":{"path":"part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet","partitionValues":{"part":"2"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":2,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":22,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"]m5]-UtmB0Rl$j","offset":1,"sizeInBytes":34,"cardinality":1}}} +{"remove":{"path":"part=8/part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet","deletionTimestamp":1688691765550,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"8"},"size":732}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000006.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000006.json new file mode 100644 index 000000000000..3b628111329c --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000006.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691770236,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#7069 = 10)\"]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"3110","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"7b578eea-e45c-4308-8c86-3c4c4f2bb9f9"}} +{"add":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","partitionValues":{"part":"0"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":0,\"col2\":\"foo0\"},\"maxValues\":{\"col1\":20,\"col2\":\"foo0\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"Zcq?bs*HKQWCB[Sjf[.o","offset":1,"sizeInBytes":36,"cardinality":2}}} +{"remove":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","deletionTimestamp":1688691769658,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"0"},"size":736,"deletionVector":{"storageType":"u","pathOrInlineDv":"0X9F0q2<2yJ-f)Gm2!e0","offset":1,"sizeInBytes":34,"cardinality":1}}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000007.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000007.json new file mode 100644 index 000000000000..cb693339679c --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000007.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691774168,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#8351 = 12)\"]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"2645","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"970ae4f8-d941-4029-b282-20e244e00cd1"}} +{"add":{"path":"part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet","partitionValues":{"part":"2"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":2,\"col2\":\"foo2\"},\"maxValues\":{\"col1\":22,\"col2\":\"foo2\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"NZQXpd#3xxK!oBujr/=:","offset":1,"sizeInBytes":36,"cardinality":2}}} +{"remove":{"path":"part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet","deletionTimestamp":1688691773564,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"2"},"size":736,"deletionVector":{"storageType":"u","pathOrInlineDv":"]m5]-UtmB0Rl$j","offset":1,"sizeInBytes":34,"cardinality":1}}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000011.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000011.json new file mode 100644 index 000000000000..11f254d86a80 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000011.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1688691788332,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#13579 = 20)\"]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"2084","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"ad17e138-a1c2-4b6a-a373-e334c138c9e0"}} +{"remove":{"path":"part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet","deletionTimestamp":1688691788291,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"0"},"size":736,"deletionVector":{"storageType":"u","pathOrInlineDv":"Zcq?bs*HKQWCB[Sjf[.o","offset":1,"sizeInBytes":36,"cardinality":2}}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000012.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000012.json new file mode 100644 index 000000000000..696d56251f1b --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000012.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1688691791603,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#15220 = 22)\"]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"1870","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"68c7ac7c-6e73-4aa4-864f-ef374e110de6"}} +{"remove":{"path":"part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet","deletionTimestamp":1688691791568,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"2"},"size":736,"deletionVector":{"storageType":"u","pathOrInlineDv":"NZQXpd#3xxK!oBujr/=:","offset":1,"sizeInBytes":36,"cardinality":2}}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000013.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000013.json new file mode 100644 index 000000000000..53759c697183 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000013.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1688691796131,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#16548 = 24)\"]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"3461","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"4e6f5590-7630-44d1-bc1c-5a2bfd637c4f"}} +{"remove":{"path":"part=4/part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet","deletionTimestamp":1688691796068,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"4"},"size":735,"deletionVector":{"storageType":"u","pathOrInlineDv":"+V9Oq([R6mTaoM.}Isa)","offset":1,"sizeInBytes":36,"cardinality":2}}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000014.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000014.json new file mode 100644 index 000000000000..e9eb0375f6b5 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000014.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691798638,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#17876 = 26)\"]"},"readVersion":13,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"1451","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"34f3a5ca-2166-4b49-a92b-f6b70ce4cd89"}} +{"add":{"path":"part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet","partitionValues":{"part":"6"},"size":736,"modificationTime":1688691720000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":26,\"col2\":\"foo1\"},\"maxValues\":{\"col1\":46,\"col2\":\"foo1\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"]g#Gi8g29gLy&KMkGJr?","offset":1,"sizeInBytes":34,"cardinality":1}}} +{"remove":{"path":"part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet","deletionTimestamp":1688691798060,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"6"},"size":736}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000015.json b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000015.json new file mode 100644 index 000000000000..c4d415d19039 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/00000000000000000015.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1688691801431,"operation":"DELETE","operationParameters":{"predicate":"[\"(col1#19379 = 28)\"]"},"readVersion":14,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"0","numRemovedBytes":"0","numCopiedRows":"0","numAddedChangeFiles":"0","executionTimeMs":"1804","numDeletedRows":"1","scanTimeMs":"0","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.4.0 Delta-Lake/3.0.0-SNAPSHOT","txnId":"ace020dc-2202-4814-bbd4-36fc7a2a7988"}} +{"add":{"path":"part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet","partitionValues":{"part":"8"},"size":736,"modificationTime":1688691721000,"dataChange":true,"stats":"{\"numRecords\":3,\"minValues\":{\"col1\":28,\"col2\":\"foo3\"},\"maxValues\":{\"col1\":48,\"col2\":\"foo3\"},\"nullCount\":{\"col1\":0,\"col2\":0},\"tightBounds\":false}","deletionVector":{"storageType":"u","pathOrInlineDv":"7NnC4LX-RqU.S:B4VD9n","offset":1,"sizeInBytes":34,"cardinality":1}}} +{"remove":{"path":"part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet","deletionTimestamp":1688691800672,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"part":"8"},"size":736}} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/_last_checkpoint b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/_last_checkpoint new file mode 100644 index 000000000000..93a5d0b0cc45 --- /dev/null +++ b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":30,"sizeInBytes":20573,"numOfAddFiles":18,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues_parsed","type":{"type":"struct","fields":[{"name":"part","type":"integer","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"822eda70bb966b38d646f351ea753463"} diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0229ec3d-5100-44e1-8e53-702d448da8da.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin new file mode 100644 index 000000000000..aa2b85703430 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_0237686e-c424-4e4c-8337-e8bd1b02ea48.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_179561f5-946c-40d8-b088-0d890e8c854c.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin new file mode 100644 index 000000000000..8421378d1645 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_2fc6c93f-f217-47db-8582-b9732a18de04.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin new file mode 100644 index 000000000000..aa2b85703430 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_3cf682dd-5194-4fe8-98ed-d860be48ef78.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_57eabe30-1981-4c70-85b0-343c24650691.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin new file mode 100644 index 000000000000..aa2b85703430 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_6f6c4302-fd0b-49e9-8877-cf9056f4b3cb.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin new file mode 100644 index 000000000000..8421378d1645 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_9ab6d39e-2b86-4282-919f-c0813c228da9.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_aa46415b-60bb-4096-a0c7-de47449cb72e.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_acb1fa71-86e9-445d-833c-5cda9a64f221.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin new file mode 100644 index 000000000000..8421378d1645 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_be3f06b1-59b3-4537-b5de-554c3bb2fad9.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin new file mode 100644 index 000000000000..8421378d1645 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_cc54e26b-ec32-4705-ab80-eccffa88e1e6.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin new file mode 100644 index 000000000000..8421378d1645 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_db0b9397-6055-4aa6-a8f8-de723719d996.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f34fad76-197a-4fd7-9382-f7773fc8eff9.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin new file mode 100644 index 000000000000..ff6b720eeb89 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/deletion_vector_f37f3d8e-af4f-40c0-a5b4-5b3c02c1bdd8.bin differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet.crc new file mode 100644 index 000000000000..ce559a08965a Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet.crc new file mode 100644 index 000000000000..c7425f64b284 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/.part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet new file mode 100644 index 000000000000..29a335170856 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00000-8387c699-30b1-4734-a791-9278d560ec19.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet new file mode 100644 index 000000000000..fd71b798fcab Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=0/part-00001-24cdbe06-d3dc-449f-bd38-575228ca42a7.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet.crc new file mode 100644 index 000000000000..dc125b5512b8 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet.crc new file mode 100644 index 000000000000..f80c2549df1d Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/.part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet new file mode 100644 index 000000000000..ad731903d5ed Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00000-a1586fa1-50e8-4f06-858a-b43b2e83010b.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet new file mode 100644 index 000000000000..c6fadc316c60 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=1/part-00001-d7e5d32a-55fa-410a-afee-adcdf46bc859.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet.crc new file mode 100644 index 000000000000..9ae113521eb3 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet.crc new file mode 100644 index 000000000000..ebe76cf833b3 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/.part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet new file mode 100644 index 000000000000..b2397fcffc70 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00000-ad58cb56-93db-4374-91ba-e65e7fa68e76.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet new file mode 100644 index 000000000000..1a78a25dee15 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=2/part-00001-ab1247be-1f77-41e6-a392-50a99b2db864.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet.crc new file mode 100644 index 000000000000..75f08be47b4a Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet.crc new file mode 100644 index 000000000000..ad539897cc22 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/.part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet new file mode 100644 index 000000000000..739317a1646a Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00000-319bea86-657f-4431-9b26-949dba99cf2c.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet new file mode 100644 index 000000000000..04099bee284d Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=3/part-00001-afeef1dd-2517-49b9-873e-e9e6e8a74b19.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet.crc new file mode 100644 index 000000000000..fccce17086ed Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet.crc new file mode 100644 index 000000000000..89c625b71cbd Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/.part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet new file mode 100644 index 000000000000..160a167dd83e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00000-69ec928d-3737-4eb3-a3d8-9555a6b55ff5.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet new file mode 100644 index 000000000000..ae085405b09b Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=4/part-00001-e63d3db6-9e97-4472-aacc-6af9fa44e73d.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet.crc new file mode 100644 index 000000000000..83b466f42d51 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet.crc new file mode 100644 index 000000000000..97f4e02462b7 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/.part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet new file mode 100644 index 000000000000..d7dca124eccc Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00000-5c963f16-d5b8-4f8b-8d8a-0e3403228be2.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet new file mode 100644 index 000000000000..06f900c77434 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=5/part-00001-f344b457-fbd0-4bc4-9502-2c07025e5bb1.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet.crc new file mode 100644 index 000000000000..2f3ac89645d5 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet.crc new file mode 100644 index 000000000000..a68eb95c2e07 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/.part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet new file mode 100644 index 000000000000..bc22316bb39e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00000-be524334-115d-4d01-8614-e1bc8c630926.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet new file mode 100644 index 000000000000..e3bffbaead5b Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=6/part-00001-6fc16401-ac51-4b89-bf08-bb86cecb5cc2.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet.crc new file mode 100644 index 000000000000..4fcb7446a0d5 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet.crc new file mode 100644 index 000000000000..7530937d86d3 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/.part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet new file mode 100644 index 000000000000..4be01dc2315e Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00000-33cc19fc-3607-4ea7-ab6d-af4e3ebf62c4.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet new file mode 100644 index 000000000000..287580ce844d Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=7/part-00001-986abb06-e672-4134-83d4-261752b236b8.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet.crc new file mode 100644 index 000000000000..91e7f0419956 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet.crc new file mode 100644 index 000000000000..4585285634af Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/.part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet new file mode 100644 index 000000000000..bbaecb8a5614 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00000-02c66988-3465-4483-9f85-7155e6aee1f4.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet new file mode 100644 index 000000000000..913958276c9a Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=8/part-00001-7c58de64-d72f-4373-8d86-dfdc00fb264e.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet.crc new file mode 100644 index 000000000000..3938aafd5fbf Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet.crc b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet.crc new file mode 100644 index 000000000000..fda17c3c3b48 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/.part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet.crc differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet new file mode 100644 index 000000000000..29a603344b0b Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00000-e4012c8c-cc60-44c0-babb-8c5d264a3a31.c000.snappy.parquet differ diff --git a/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet new file mode 100644 index 000000000000..bb97b2a643d3 Binary files /dev/null and b/delta-lake/src/test/resources/delta/golden/dv-partitioned-with-checkpoint/part=9/part-00001-c0430af8-a8e0-4b23-8776-b2fc549b3e4e.c000.snappy.parquet differ diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 52a800723598..27816b024340 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -29,3 +29,4 @@ sitemap.xml .python-version **/*_index.md **/.venv/** +**/_delta_log/_last_checkpoint